You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by md...@apache.org on 2014/04/22 01:48:12 UTC

[10/11] ACCUMULO-1880 create mapreduce module

http://git-wip-us.apache.org/repos/asf/accumulo/blob/99baad37/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
deleted file mode 100644
index 0cee355..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ /dev/null
@@ -1,383 +0,0 @@
-/*
- * 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.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-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.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.Pair;
-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;
-
-/**
- * This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of K,V pairs.
- * <p>
- * Subclasses must implement a {@link #getRecordReader(InputSplit, JobConf, Reporter)} to provide a {@link RecordReader} for K,V.
- * <p>
- * A static base class, RecordReaderBase, is provided to retrieve Accumulo {@link Key}/{@link Value} pairs, but one must implement its
- * {@link RecordReaderBase#next(Object, Object)} to transform them to the desired generic types K,V.
- * <p>
- * See {@link AccumuloInputFormat} for an example implementation.
- */
-public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
-
-  /**
-   * Sets the name of the input table, over which this job will scan.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param tableName
-   *          the table to use when the tablename is null in the write call
-   * @since 1.5.0
-   */
-  public static void setInputTableName(JobConf job, String tableName) {
-    InputConfigurator.setInputTableName(CLASS, job, tableName);
-  }
-
-  /**
-   * Gets the table name from the configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the table name
-   * @since 1.5.0
-   * @see #setInputTableName(JobConf, String)
-   */
-  protected static String getInputTableName(JobConf job) {
-    return InputConfigurator.getInputTableName(CLASS, job);
-  }
-
-  /**
-   * Sets the input ranges to scan for this job. If not set, the entire table will be scanned.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param ranges
-   *          the ranges that will be mapped over
-   * @since 1.5.0
-   */
-  public static void setRanges(JobConf job, Collection<Range> ranges) {
-    InputConfigurator.setRanges(CLASS, job, ranges);
-  }
-
-  /**
-   * Gets the ranges to scan over from a job.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the ranges
-   * @throws IOException
-   *           if the ranges have been encoded improperly
-   * @since 1.5.0
-   * @see #setRanges(JobConf, Collection)
-   */
-  protected static List<Range> getRanges(JobConf job) throws IOException {
-    return InputConfigurator.getRanges(CLASS, job);
-  }
-
-  /**
-   * Restricts the columns that will be mapped over for this job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param columnFamilyColumnQualifierPairs
-   *          a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
-   *          selected. An empty set is the default and is equivalent to scanning the all columns.
-   * @since 1.5.0
-   */
-  public static void fetchColumns(JobConf job, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    InputConfigurator.fetchColumns(CLASS, job, columnFamilyColumnQualifierPairs);
-  }
-
-  /**
-   * Gets the columns to be mapped over from this job.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return a set of columns
-   * @since 1.5.0
-   * @see #fetchColumns(JobConf, Collection)
-   */
-  protected static Set<Pair<Text,Text>> getFetchedColumns(JobConf job) {
-    return InputConfigurator.getFetchedColumns(CLASS, job);
-  }
-
-  /**
-   * Encode an iterator on the input for this job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param cfg
-   *          the configuration of the iterator
-   * @since 1.5.0
-   */
-  public static void addIterator(JobConf job, IteratorSetting cfg) {
-    InputConfigurator.addIterator(CLASS, job, cfg);
-  }
-
-  /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return a list of iterators
-   * @since 1.5.0
-   * @see #addIterator(JobConf, IteratorSetting)
-   */
-  protected static List<IteratorSetting> getIterators(JobConf job) {
-    return InputConfigurator.getIterators(CLASS, job);
-  }
-
-  /**
-   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
-   * Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
-   * 
-   * <p>
-   * By default, this feature is <b>enabled</b>.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @see #setRanges(JobConf, Collection)
-   * @since 1.5.0
-   */
-  public static void setAutoAdjustRanges(JobConf job, boolean enableFeature) {
-    InputConfigurator.setAutoAdjustRanges(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has auto-adjust ranges enabled.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return false if the feature is disabled, true otherwise
-   * @since 1.5.0
-   * @see #setAutoAdjustRanges(JobConf, boolean)
-   */
-  protected static boolean getAutoAdjustRanges(JobConf job) {
-    return InputConfigurator.getAutoAdjustRanges(CLASS, job);
-  }
-
-  /**
-   * Controls the use of the {@link IsolatedScanner} in this job.
-   * 
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setScanIsolation(JobConf job, boolean enableFeature) {
-    InputConfigurator.setScanIsolation(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has isolation enabled.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setScanIsolation(JobConf, boolean)
-   */
-  protected static boolean isIsolated(JobConf job) {
-    return InputConfigurator.isIsolated(CLASS, job);
-  }
-
-  /**
-   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack to be constructed within the Map
-   * task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be available on the classpath for the task.
-   * 
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setLocalIterators(JobConf job, boolean enableFeature) {
-    InputConfigurator.setLocalIterators(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration uses local iterators.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setLocalIterators(JobConf, boolean)
-   */
-  protected static boolean usesLocalIterators(JobConf job) {
-    return InputConfigurator.usesLocalIterators(CLASS, job);
-  }
-
-  /**
-   * <p>
-   * Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
-   * table's files. If the table is not offline, then the job will fail. If the table comes online during the map reduce job, it is likely that the job will
-   * fail.
-   * 
-   * <p>
-   * To use this option, the map reduce user will need access to read the Accumulo directory in HDFS.
-   * 
-   * <p>
-   * Reading the offline table will create the scan time iterator stack in the map process. So any iterators that are configured for the table will need to be
-   * on the mapper's classpath.
-   * 
-   * <p>
-   * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
-   * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
-   * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
-   * 
-   * <p>
-   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
-   * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
-   * 
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setOfflineTableScan(JobConf job, boolean enableFeature) {
-    InputConfigurator.setOfflineTableScan(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the offline table scan feature enabled.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setOfflineTableScan(JobConf, boolean)
-   */
-  protected static boolean isOfflineScan(JobConf job) {
-    return InputConfigurator.isOfflineScan(CLASS, job);
-  }
-
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
-   * 
-   * @param job
-   *          the Hadoop job for the configured job
-   * @return an Accumulo tablet locator
-   * @throws org.apache.accumulo.core.client.TableNotFoundException
-   *           if the table name set on the job doesn't exist
-   * @since 1.5.0
-   * @deprecated since 1.6.0
-   */
-  @Deprecated
-  protected static TabletLocator getTabletLocator(JobConf job) throws TableNotFoundException {
-    return InputConfigurator.getTabletLocator(CLASS, job, InputConfigurator.getInputTableName(CLASS, job));
-  }
-
-  protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
-
-    @Override
-    protected void setupIterators(JobConf job, Scanner scanner, String tableName, org.apache.accumulo.core.client.mapred.RangeInputSplit split) {
-      List<IteratorSetting> iterators = null;
-
-      if (null == split) {
-        iterators = getIterators(job);
-      } else {
-        iterators = split.getIterators();
-        if (null == iterators) {
-          iterators = getIterators(job);
-        }
-      }
-
-      setupIterators(iterators, scanner);
-    }
-
-    /**
-     * Apply the configured iterators to the scanner.
-     * 
-     * @param iterators
-     *          the iterators to set
-     * @param scanner
-     *          the scanner to configure
-     */
-    protected void setupIterators(List<IteratorSetting> iterators, Scanner scanner) {
-      for (IteratorSetting iterator : iterators) {
-        scanner.addScanIterator(iterator);
-      }
-    }
-
-    /**
-     * Apply the configured iterators from the configuration to the scanner.
-     * 
-     * @param job
-     *          the job configuration
-     * @param scanner
-     *          the scanner to configure
-     */
-    @Deprecated
-    protected void setupIterators(JobConf job, Scanner scanner) {
-      setupIterators(getIterators(job), scanner);
-    }
-  }
-
-  /**
-   * @deprecated since 1.5.2; Use {@link org.apache.accumulo.core.client.mapred.RangeInputSplit} instead.
-   * @see org.apache.accumulo.core.client.mapred.RangeInputSplit
-   */
-  @Deprecated
-  public static class RangeInputSplit extends org.apache.accumulo.core.client.mapred.RangeInputSplit {
-    public RangeInputSplit() {
-      super();
-    }
-
-    public RangeInputSplit(RangeInputSplit other) throws IOException {
-      super(other);
-    }
-
-    public RangeInputSplit(String table, String tableId, Range range, String[] locations) {
-      super(table, tableId, range, locations);
-    }
-
-    protected RangeInputSplit(String table, Range range, String[] locations) {
-      super(table, "", range, locations);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/99baad37/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
deleted file mode 100644
index 3fd2ab0..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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 org.apache.accumulo.core.data.Range;
-import org.apache.hadoop.mapred.InputSplit;
-
-/**
- * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
- */
-public class RangeInputSplit extends org.apache.accumulo.core.client.mapreduce.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/99baad37/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
deleted file mode 100644
index 836cff9..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ /dev/null
@@ -1,672 +0,0 @@
-/*
- * 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.IOException;
-import java.lang.reflect.Method;
-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.Random;
-
-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.impl.InputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
-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.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
-   * @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
-   * @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 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, ClientConfiguration)
-   * @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, 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 = 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: " + table);
-        log.debug("Authorizations are: " + authorizations);
-        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 (isIsolated) {
-          log.info("Creating isolated scanner");
-          scanner = new IsolatedScanner(scanner);
-        }
-        if (usesLocalIterators) {
-          log.info("Using local iterators");
-          scanner = new ClientSideIteratorScanner(scanner);
-        }
-
-        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 : 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);
-    }
-
-    /**
-     * The Key that should be returned to the client
-     */
-    protected K currentK = null;
-
-    /**
-     * The Value that should be return to the client
-     */
-    protected V currentV = null;
-
-    /**
-     * The Key that is used to determine progress in the current InputSplit. It is not returned to the client and is only used internally
-     */
-    protected Key currentKey = 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
-   */
-  @Override
-  public List<InputSplit> getSplits(JobContext context) throws IOException {
-    Level logLevel = getLogLevel(context);
-    log.setLevel(logLevel);
-    validateOptions(context);
-    Random random = new Random();
-    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();
-      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(context, tableId, ranges);
-          while (binnedRanges == null) {
-            // Some tablets were still online, try again
-            UtilWaitThread.sleep(100 + random.nextInt(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 + random.nextInt(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.getCanonicalHostName();
-          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
-              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()) {
-          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);
-        }
-    }
-    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/99baad37/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
deleted file mode 100644
index 196fb04..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * 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.IOException;
-import java.util.Arrays;
-
-import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.ArrayByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.commons.collections.map.LRUMap;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to write output in the Accumulo data file format.<br />
- * Care should be taken to write only sorted data (sorted by {@link Key}), as this is an important requirement of Accumulo data files.
- * 
- * <p>
- * The output path to be created must be specified via {@link AccumuloFileOutputFormat#setOutputPath(Job, Path)}. This is inherited from
- * {@link FileOutputFormat#setOutputPath(Job, Path)}. Other methods from {@link FileOutputFormat} are not supported and may be ignored or cause failures. Using
- * other Hadoop configuration options that affect the behavior of the underlying files directly in the Job's configuration may work, but are not directly
- * supported at this time.
- */
-public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
-  
-  private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-  
-  /**
-   * This helper method provides an AccumuloConfiguration object constructed from the Accumulo defaults, and overridden with Accumulo properties that have been
-   * stored in the Job's configuration.
-   * 
-   * @param context
-   *          the Hadoop context for the configured job
-   * @since 1.5.0
-   */
-  protected static AccumuloConfiguration getAccumuloConfiguration(JobContext context) {
-    return FileOutputConfigurator.getAccumuloConfiguration(CLASS, InputFormatBase.getConfiguration(context));
-  }
-  
-  /**
-   * Sets the compression type to use for data blocks. Specifying a compression may require additional libraries to be available to your Job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param compressionType
-   *          one of "none", "gz", "lzo", or "snappy"
-   * @since 1.5.0
-   */
-  public static void setCompressionType(Job job, String compressionType) {
-    FileOutputConfigurator.setCompressionType(CLASS, job.getConfiguration(), compressionType);
-  }
-  
-  /**
-   * Sets the size for data blocks within each file.<br />
-   * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as a group.
-   * 
-   * <p>
-   * Making this value smaller may increase seek performance, but at the cost of increasing the size of the indexes (which can also affect seek performance).
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param dataBlockSize
-   *          the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setDataBlockSize(Job job, long dataBlockSize) {
-    FileOutputConfigurator.setDataBlockSize(CLASS, job.getConfiguration(), dataBlockSize);
-  }
-  
-  /**
-   * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by the underlying file system.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param fileBlockSize
-   *          the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setFileBlockSize(Job job, long fileBlockSize) {
-    FileOutputConfigurator.setFileBlockSize(CLASS, job.getConfiguration(), fileBlockSize);
-  }
-  
-  /**
-   * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy within the file, while larger blocks mean a more shallow
-   * index hierarchy within the file. This can affect the performance of queries.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param indexBlockSize
-   *          the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setIndexBlockSize(Job job, long indexBlockSize) {
-    FileOutputConfigurator.setIndexBlockSize(CLASS, job.getConfiguration(), indexBlockSize);
-  }
-  
-  /**
-   * Sets the file system replication factor for the resulting file, overriding the file system default.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param replication
-   *          the number of replicas for produced files
-   * @since 1.5.0
-   */
-  public static void setReplication(Job job, int replication) {
-    FileOutputConfigurator.setReplication(CLASS, job.getConfiguration(), replication);
-  }
-  
-  @Override
-  public RecordWriter<Key,Value> getRecordWriter(TaskAttemptContext context) throws IOException {
-    // get the path of the temporary output file
-    final Configuration conf = InputFormatBase.getConfiguration(context);
-    final AccumuloConfiguration acuConf = getAccumuloConfiguration(context);
-    
-    final String extension = acuConf.get(Property.TABLE_FILE_TYPE);
-    final Path file = this.getDefaultWorkFile(context, "." + extension);
-    
-    final LRUMap validVisibilities = new LRUMap(1000);
-    
-    return new RecordWriter<Key,Value>() {
-      FileSKVWriter out = null;
-      
-      @Override
-      public void close(TaskAttemptContext context) throws IOException {
-        if (out != null)
-          out.close();
-      }
-      
-      @Override
-      public void write(Key key, Value value) throws IOException {
-        
-        Boolean wasChecked = (Boolean) validVisibilities.get(key.getColumnVisibilityData());
-        if (wasChecked == null) {
-          byte[] cv = key.getColumnVisibilityData().toArray();
-          new ColumnVisibility(cv);
-          validVisibilities.put(new ArrayByteSequence(Arrays.copyOf(cv, cv.length)), Boolean.TRUE);
-        }
-        
-        if (out == null) {
-          out = FileOperations.getInstance().openWriter(file.toString(), file.getFileSystem(conf), conf, acuConf);
-          out.startDefaultLocalityGroup();
-        }
-        out.append(key, value);
-      }
-    };
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/99baad37/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
deleted file mode 100644
index 21a0280..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.ClientConfiguration;
-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.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.log4j.Level;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link 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 AccumuloInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloInputFormat#setScanAuthorizations(Job, Authorizations)}
- * <li>{@link AccumuloInputFormat#setZooKeeperInstance(Job, ClientConfiguration)} OR {@link AccumuloInputFormat#setMockInstance(Job, String)}
- * </ul>
- * 
- * Other static methods are optional.
- */
-public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
-
-  @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 org.apache.accumulo.core.client.mapreduce.RangeInputSplit) {
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit risplit = (org.apache.accumulo.core.client.mapreduce.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()) {
-          ++numKeysRead;
-          Entry<Key,Value> entry = scannerIterator.next();
-          currentK = currentKey = entry.getKey();
-          currentV = entry.getValue();
-          if (log.isTraceEnabled())
-            log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
-          return true;
-        }
-        return false;
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/99baad37/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
deleted file mode 100644
index af1001f..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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.impl.InputConfigurator;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-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 = 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, 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/99baad37/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
deleted file mode 100644
index af9bbae..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ /dev/null
@@ -1,545 +0,0 @@
-/*
- * 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.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-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.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.MultiTableBatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.SecurityErrorCode;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the sink for data. This {@link OutputFormat} accepts keys and values of type {@link Text} (for a table
- * name) and {@link Mutation} from the Map and Reduce functions.
- * 
- * The user must specify the following via static configurator methods:
- * 
- * <ul>
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(Job, ClientConfiguration)} OR {@link AccumuloOutputFormat#setMockInstance(Job, String)}
- * </ul>
- * 
- * Other static methods are optional.
- */
-public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
-
-  private static final Class<?> CLASS = AccumuloOutputFormat.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 if {@link #setCreateTables(Job, boolean)} is set to true)
-   * @param token
-   *          the user's password
-   * @since 1.5.0
-   */
-  public static void setConnectorInfo(Job job, String principal, AuthenticationToken token) throws AccumuloSecurityException {
-    OutputConfigurator.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 if {@link #setCreateTables(Job, boolean)} is set to true)
-   * @param tokenFile
-   *          the path to the token file
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(Job job, String principal, String tokenFile) throws AccumuloSecurityException {
-    OutputConfigurator.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 OutputConfigurator.isConnectorInfoSet(CLASS, InputFormatBase.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 OutputConfigurator.getPrincipal(CLASS, InputFormatBase.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 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 OutputConfigurator.getAuthenticationToken(CLASS, InputFormatBase.getConfiguration(context));
-  }
-
-  /**
-   * Configures a {@link ZooKeeperInstance} for this job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param clientConfig
-   *          client configuration for specifying connection timeouts, SSL connection options, etc.
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(Job job, ClientConfiguration clientConfig) {
-    OutputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
-  }
-
-  /**
-   * 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(Job job, String instanceName) {
-    OutputConfigurator.setMockInstance(CLASS, job.getConfiguration(), instanceName);
-  }
-
-  /**
-   * Initializes an Accumulo {@link 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, ClientConfiguration)
-   * @see #setMockInstance(Job, String)
-   */
-  protected static Instance getInstance(JobContext context) {
-    return OutputConfigurator.getInstance(CLASS, InputFormatBase.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) {
-    OutputConfigurator.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 OutputConfigurator.getLogLevel(CLASS, InputFormatBase.getConfiguration(context));
-  }
-
-  /**
-   * Sets the default table name to use if one emits a null in place of a table name for a given mutation. Table names can only be alpha-numeric and
-   * underscores.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param tableName
-   *          the table to use when the tablename is null in the write call
-   * @since 1.5.0
-   */
-  public static void setDefaultTableName(Job job, String tableName) {
-    OutputConfigurator.setDefaultTableName(CLASS, job.getConfiguration(), tableName);
-  }
-
-  /**
-   * Gets the default table name from the configuration.
-   * 
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return the default table name
-   * @since 1.5.0
-   * @see #setDefaultTableName(Job, String)
-   */
-  protected static String getDefaultTableName(JobContext context) {
-    return OutputConfigurator.getDefaultTableName(CLASS, InputFormatBase.getConfiguration(context));
-  }
-
-  /**
-   * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new {@link BatchWriterConfig}, with sensible built-in defaults is
-   * used. Setting the configuration multiple times overwrites any previous configuration.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param bwConfig
-   *          the configuration for the {@link BatchWriter}
-   * @since 1.5.0
-   */
-  public static void setBatchWriterOptions(Job job, BatchWriterConfig bwConfig) {
-    OutputConfigurator.setBatchWriterOptions(CLASS, job.getConfiguration(), bwConfig);
-  }
-
-  /**
-   * Gets the {@link BatchWriterConfig} settings.
-   * 
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return the configuration object
-   * @since 1.5.0
-   * @see #setBatchWriterOptions(Job, BatchWriterConfig)
-   */
-  protected static BatchWriterConfig getBatchWriterOptions(JobContext context) {
-    return OutputConfigurator.getBatchWriterOptions(CLASS, InputFormatBase.getConfiguration(context));
-  }
-
-  /**
-   * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric and underscores.
-   * 
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setCreateTables(Job job, boolean enableFeature) {
-    OutputConfigurator.setCreateTables(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether tables are permitted to be created as needed.
-   * 
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return true if the feature is disabled, false otherwise
-   * @since 1.5.0
-   * @see #setCreateTables(Job, boolean)
-   */
-  protected static Boolean canCreateTables(JobContext context) {
-    return OutputConfigurator.canCreateTables(CLASS, InputFormatBase.getConfiguration(context));
-  }
-
-  /**
-   * Sets the directive to use simulation mode for this job. In simulation mode, no output is produced. This is useful for testing.
-   * 
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setSimulationMode(Job job, boolean enableFeature) {
-    OutputConfigurator.setSimulationMode(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether this feature is enabled.
-   * 
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setSimulationMode(Job, boolean)
-   */
-  protected static Boolean getSimulationMode(JobContext context) {
-    return OutputConfigurator.getSimulationMode(CLASS, InputFormatBase.getConfiguration(context));
-  }
-
-  /**
-   * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
-   */
-  protected static class AccumuloRecordWriter extends RecordWriter<Text,Mutation> {
-    private MultiTableBatchWriter mtbw = null;
-    private HashMap<Text,BatchWriter> bws = null;
-    private Text defaultTableName = null;
-
-    private boolean simulate = false;
-    private boolean createTables = false;
-
-    private long mutCount = 0;
-    private long valCount = 0;
-
-    private Connector conn;
-
-    protected AccumuloRecordWriter(TaskAttemptContext context) throws AccumuloException, AccumuloSecurityException, IOException {
-      Level l = getLogLevel(context);
-      if (l != null)
-        log.setLevel(getLogLevel(context));
-      this.simulate = getSimulationMode(context);
-      this.createTables = canCreateTables(context);
-
-      if (simulate)
-        log.info("Simulating output only. No writes to tables will occur");
-
-      this.bws = new HashMap<Text,BatchWriter>();
-
-      String tname = getDefaultTableName(context);
-      this.defaultTableName = (tname == null) ? null : new Text(tname);
-
-      if (!simulate) {
-        this.conn = getInstance(context).getConnector(getPrincipal(context), getAuthenticationToken(context));
-        mtbw = conn.createMultiTableBatchWriter(getBatchWriterOptions(context));
-      }
-    }
-
-    /**
-     * Push a mutation into a table. If table is null, the defaultTable will be used. If canCreateTable is set, the table will be created if it does not exist.
-     * The table name must only contain alphanumerics and underscore.
-     */
-    @Override
-    public void write(Text table, Mutation mutation) throws IOException {
-      if (table == null || table.toString().isEmpty())
-        table = this.defaultTableName;
-
-      if (!simulate && table == null)
-        throw new IOException("No table or default table specified. Try simulation mode next time");
-
-      ++mutCount;
-      valCount += mutation.size();
-      printMutation(table, mutation);
-
-      if (simulate)
-        return;
-
-      if (!bws.containsKey(table))
-        try {
-          addTable(table);
-        } catch (Exception e) {
-          e.printStackTrace();
-          throw new IOException(e);
-        }
-
-      try {
-        bws.get(table).addMutation(mutation);
-      } catch (MutationsRejectedException e) {
-        throw new IOException(e);
-      }
-    }
-
-    public void addTable(Text tableName) throws AccumuloException, AccumuloSecurityException {
-      if (simulate) {
-        log.info("Simulating adding table: " + tableName);
-        return;
-      }
-
-      log.debug("Adding table: " + tableName);
-      BatchWriter bw = null;
-      String table = tableName.toString();
-
-      if (createTables && !conn.tableOperations().exists(table)) {
-        try {
-          conn.tableOperations().create(table);
-        } catch (AccumuloSecurityException e) {
-          log.error("Accumulo security violation creating " + table, e);
-          throw e;
-        } catch (TableExistsException e) {
-          // Shouldn't happen
-        }
-      }
-
-      try {
-        bw = mtbw.getBatchWriter(table);
-      } catch (TableNotFoundException e) {
-        log.error("Accumulo table " + table + " doesn't exist and cannot be created.", e);
-        throw new AccumuloException(e);
-      } catch (AccumuloException e) {
-        throw e;
-      } catch (AccumuloSecurityException e) {
-        throw e;
-      }
-
-      if (bw != null)
-        bws.put(tableName, bw);
-    }
-
-    private int printMutation(Text table, Mutation m) {
-      if (log.isTraceEnabled()) {
-        log.trace(String.format("Table %s row key: %s", table, hexDump(m.getRow())));
-        for (ColumnUpdate cu : m.getUpdates()) {
-          log.trace(String.format("Table %s column: %s:%s", table, hexDump(cu.getColumnFamily()), hexDump(cu.getColumnQualifier())));
-          log.trace(String.format("Table %s security: %s", table, new ColumnVisibility(cu.getColumnVisibility()).toString()));
-          log.trace(String.format("Table %s value: %s", table, hexDump(cu.getValue())));
-        }
-      }
-      return m.getUpdates().size();
-    }
-
-    private String hexDump(byte[] ba) {
-      StringBuilder sb = new StringBuilder();
-      for (byte b : ba) {
-        if ((b > 0x20) && (b < 0x7e))
-          sb.append((char) b);
-        else
-          sb.append(String.format("x%02x", b));
-      }
-      return sb.toString();
-    }
-
-    @Override
-    public void close(TaskAttemptContext attempt) throws IOException, InterruptedException {
-      log.debug("mutations written: " + mutCount + ", values written: " + valCount);
-      if (simulate)
-        return;
-
-      try {
-        mtbw.close();
-      } catch (MutationsRejectedException e) {
-        if (e.getAuthorizationFailuresMap().size() >= 0) {
-          HashMap<String,Set<SecurityErrorCode>> tables = new HashMap<String,Set<SecurityErrorCode>>();
-          for (Entry<KeyExtent,Set<SecurityErrorCode>> ke : e.getAuthorizationFailuresMap().entrySet()) {
-            Set<SecurityErrorCode> secCodes = tables.get(ke.getKey().getTableId().toString());
-            if (secCodes == null) {
-              secCodes = new HashSet<SecurityErrorCode>();
-              tables.put(ke.getKey().getTableId().toString(), secCodes);
-            }
-            secCodes.addAll(ke.getValue());
-          }
-
-          log.error("Not authorized to write to tables : " + tables);
-        }
-
-        if (e.getConstraintViolationSummaries().size() > 0) {
-          log.error("Constraint violations : " + e.getConstraintViolationSummaries().size());
-        }
-      }
-    }
-  }
-
-  @Override
-  public void checkOutputSpecs(JobContext job) throws IOException {
-    if (!isConnectorInfoSet(job))
-      throw new IOException("Connector info has not been set.");
-    try {
-      // if the instance isn't configured, it will complain here
-      String principal = getPrincipal(job);
-      AuthenticationToken token = getAuthenticationToken(job);
-      Connector c = getInstance(job).getConnector(principal, token);
-      if (!c.securityOperations().authenticateUser(principal, token))
-        throw new IOException("Unable to authenticate user");
-    } catch (AccumuloException e) {
-      throw new IOException(e);
-    } catch (AccumuloSecurityException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
-  public OutputCommitter getOutputCommitter(TaskAttemptContext context) {
-    return new NullOutputFormat<Text,Mutation>().getOutputCommitter(context);
-  }
-
-  @Override
-  public RecordWriter<Text,Mutation> getRecordWriter(TaskAttemptContext attempt) throws IOException {
-    try {
-      return new AccumuloRecordWriter(attempt);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/99baad37/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
deleted file mode 100644
index 37caf15..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.RowIterator;
-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.PeekingIterator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputFormat;
-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 Accumulo as the source of data. This {@link InputFormat} provides row names as {@link Text} as keys, and a
- * corresponding {@link PeekingIterator} as a value, which in turn makes the {@link Key}/{@link Value} pairs for that row available to the Map function.
- * 
- * The user must specify the following via static configurator methods:
- * 
- * <ul>
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloRowInputFormat#setInputTableName(Job, String)}
- * <li>{@link AccumuloRowInputFormat#setScanAuthorizations(Job, Authorizations)}
- * <li>{@link AccumuloRowInputFormat#setZooKeeperInstance(Job, ClientConfiguration)} OR {@link AccumuloRowInputFormat#setMockInstance(Job, String)}
- * </ul>
- * 
- * Other static methods are optional.
- */
-public class AccumuloRowInputFormat extends InputFormatBase<Text,PeekingIterator<Entry<Key,Value>>> {
-  @Override
-  public RecordReader<Text,PeekingIterator<Entry<Key,Value>>> createRecordReader(InputSplit split, TaskAttemptContext context) throws IOException,
-      InterruptedException {
-    log.setLevel(getLogLevel(context));
-    return new RecordReaderBase<Text,PeekingIterator<Entry<Key,Value>>>() {
-      RowIterator rowIterator;
-
-      @Override
-      public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
-        super.initialize(inSplit, attempt);
-        rowIterator = new RowIterator(scannerIterator);
-        currentK = new Text();
-        currentV = null;
-      }
-
-      @Override
-      public boolean nextKeyValue() throws IOException, InterruptedException {
-        if (!rowIterator.hasNext())
-          return false;
-        currentV = new PeekingIterator<Entry<Key,Value>>(rowIterator.next());
-        numKeysRead = rowIterator.getKVCount();
-        currentKey = currentV.peek().getKey();
-        currentK = new Text(currentKey.getRow());
-        return true;
-      }
-    };
-  }
-}