accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mmil...@apache.org
Subject [accumulo] branch master updated: Clean up new Map Reduce internals (#884)
Date Thu, 10 Jan 2019 19:53:43 GMT
This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/master by this push:
     new 05d564f  Clean up new Map Reduce internals (#884)
05d564f is described below

commit 05d564fce5e70a4cf0de5405e4eff265419a5afe
Author: Mike Miller <mmiller@apache.org>
AuthorDate: Thu Jan 10 14:53:38 2019 -0500

    Clean up new Map Reduce internals (#884)
    
    * Create AccumuloRecordReader from AbstractRecordReader
    * Remove intermediate abstract Input classes
    * Make AccumuloRecordWriter its own class
    * Cleanup AccumuloFileOutputFormat classes
    * Drop AccumuloFileOutputFormatImpl and call configurator directly
    * Make classes pass class to job serialization
---
 .../hadoop/mapred/AccumuloFileOutputFormat.java    |   3 +-
 .../hadoop/mapred/AccumuloInputFormat.java         |   7 +-
 .../hadoop/mapred/AccumuloOutputFormat.java        |  10 +-
 .../hadoop/mapred/AccumuloRowInputFormat.java      |  11 +-
 .../hadoop/mapreduce/AccumuloFileOutputFormat.java |   3 +-
 .../hadoop/mapreduce/AccumuloInputFormat.java      |   7 +-
 .../hadoop/mapreduce/AccumuloOutputFormat.java     |  10 +-
 .../hadoop/mapreduce/AccumuloRowInputFormat.java   |   7 +-
 .../hadoopImpl/mapred/AbstractInputFormat.java     | 568 --------------------
 .../mapred/AccumuloFileOutputFormatImpl.java       | 135 -----
 .../mapred/AccumuloOutputFormatImpl.java           | 342 ------------
 .../hadoopImpl/mapred/AccumuloRecordReader.java    | 451 ++++++++++++++++
 .../hadoopImpl/mapred/AccumuloRecordWriter.java    | 212 ++++++++
 .../hadoopImpl/mapred/InputFormatBase.java         | 333 ------------
 .../hadoopImpl/mapreduce/AbstractInputFormat.java  | 587 ---------------------
 .../mapreduce/AccumuloFileOutputFormatImpl.java    | 134 -----
 .../mapreduce/AccumuloOutputFormatImpl.java        | 344 ------------
 .../hadoopImpl/mapreduce/AccumuloRecordReader.java | 479 +++++++++++++++++
 .../hadoopImpl/mapreduce/AccumuloRecordWriter.java | 213 ++++++++
 .../mapreduce/FileOutputFormatBuilderImpl.java     |  59 +--
 .../hadoopImpl/mapreduce/InputFormatBase.java      | 325 ------------
 .../mapreduce/InputFormatBuilderImpl.java          |  83 +--
 .../hadoopImpl/mapreduce/InputTableConfig.java     |   4 -
 .../mapreduce/OutputFormatBuilderImpl.java         |  35 +-
 24 files changed, 1444 insertions(+), 2918 deletions(-)

diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormat.java
index 0edcd84..016d453 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloFileOutputFormat.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.util.Progressable;
  * @since 2.0
  */
 public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
+  private static final Class<AccumuloFileOutputFormat> CLASS = AccumuloFileOutputFormat.class;
 
   @Override
   public RecordWriter<Key,Value> getRecordWriter(FileSystem ignored, JobConf job, String name,
@@ -82,7 +83,7 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
    * Sets all the information required for this map reduce job.
    */
   public static FileOutputFormatBuilder.PathParams<JobConf> configure() {
-    return new FileOutputFormatBuilderImpl<JobConf>();
+    return new FileOutputFormatBuilderImpl<>(CLASS);
   }
 
 }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormat.java
index 2523a0d..aef3237 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloInputFormat.java
@@ -23,8 +23,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.format.DefaultFormatter;
 import org.apache.accumulo.hadoop.mapreduce.InputFormatBuilder;
-import org.apache.accumulo.hadoopImpl.mapred.AbstractInputFormat;
-import org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.RecordReaderBase;
+import org.apache.accumulo.hadoopImpl.mapred.AccumuloRecordReader;
 import org.apache.accumulo.hadoopImpl.mapreduce.InputFormatBuilderImpl;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
@@ -54,14 +53,14 @@ public class AccumuloInputFormat implements InputFormat<Key,Value> {
    */
   @Override
   public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
-    return AbstractInputFormat.getSplits(job);
+    return AccumuloRecordReader.getSplits(job, CLASS);
   }
 
   @Override
   public RecordReader<Key,Value> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
       throws IOException {
 
-    RecordReaderBase<Key,Value> recordReader = new RecordReaderBase<Key,Value>() {
+    AccumuloRecordReader<Key,Value> recordReader = new AccumuloRecordReader<Key,Value>(CLASS) {
 
       @Override
       public boolean next(Key key, Value value) {
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java
index d29ed16..2386081 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java
@@ -26,8 +26,9 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.hadoop.mapreduce.OutputFormatBuilder;
-import org.apache.accumulo.hadoopImpl.mapred.AccumuloOutputFormatImpl;
+import org.apache.accumulo.hadoopImpl.mapred.AccumuloRecordWriter;
 import org.apache.accumulo.hadoopImpl.mapreduce.OutputFormatBuilderImpl;
+import org.apache.accumulo.hadoopImpl.mapreduce.lib.OutputConfigurator;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
@@ -41,10 +42,11 @@ import org.apache.hadoop.util.Progressable;
  * @since 2.0
  */
 public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
+  private static final Class<AccumuloOutputFormat> CLASS = AccumuloOutputFormat.class;
 
   @Override
   public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
-    ClientInfo clientInfo = AccumuloOutputFormatImpl.getClientInfo(job);
+    ClientInfo clientInfo = OutputConfigurator.getClientInfo(CLASS, job);
     String principal = clientInfo.getPrincipal();
     AuthenticationToken token = clientInfo.getAuthenticationToken();
     try (AccumuloClient c = Accumulo.newClient().from(clientInfo.getProperties()).build()) {
@@ -59,13 +61,13 @@ public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
   public RecordWriter<Text,Mutation> getRecordWriter(FileSystem ignored, JobConf job, String name,
       Progressable progress) throws IOException {
     try {
-      return new AccumuloOutputFormatImpl.AccumuloRecordWriter(job);
+      return new AccumuloRecordWriter(job);
     } catch (Exception e) {
       throw new IOException(e);
     }
   }
 
   public static OutputFormatBuilder.ClientParams<JobConf> configure() {
-    return new OutputFormatBuilderImpl<JobConf>();
+    return new OutputFormatBuilderImpl<>(CLASS);
   }
 }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloRowInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloRowInputFormat.java
index cb1d650..29a6d31 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloRowInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloRowInputFormat.java
@@ -24,8 +24,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.PeekingIterator;
 import org.apache.accumulo.hadoop.mapreduce.InputFormatBuilder;
-import org.apache.accumulo.hadoopImpl.mapred.AbstractInputFormat;
-import org.apache.accumulo.hadoopImpl.mapred.InputFormatBase;
+import org.apache.accumulo.hadoopImpl.mapred.AccumuloRecordReader;
 import org.apache.accumulo.hadoopImpl.mapreduce.InputFormatBuilderImpl;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.InputFormat;
@@ -53,15 +52,15 @@ public class AccumuloRowInputFormat implements InputFormat<Text,PeekingIterator<
    */
   @Override
   public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
-    return AbstractInputFormat.getSplits(job);
+    return AccumuloRecordReader.getSplits(job, CLASS);
   }
 
   @Override
   public RecordReader<Text,PeekingIterator<Entry<Key,Value>>> getRecordReader(InputSplit split,
       JobConf job, Reporter reporter) throws IOException {
     // @formatter:off
-    InputFormatBase.RecordReaderBase<Text,PeekingIterator<Entry<Key,Value>>> recordReader =
-      new InputFormatBase.RecordReaderBase<Text,PeekingIterator<Entry<Key,Value>>>() {
+    AccumuloRecordReader<Text,PeekingIterator<Entry<Key,Value>>> recordReader =
+      new AccumuloRecordReader<Text,PeekingIterator<Entry<Key,Value>>>(CLASS) {
     // @formatter:on
           RowIterator rowIterator;
 
@@ -99,6 +98,6 @@ public class AccumuloRowInputFormat implements InputFormat<Text,PeekingIterator<
    * Sets all the information required for this map reduce job.
    */
   public static InputFormatBuilder.ClientParams<JobConf> configure() {
-    return new InputFormatBuilderImpl<JobConf>(CLASS);
+    return new InputFormatBuilderImpl<>(CLASS);
   }
 }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloFileOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloFileOutputFormat.java
index 7e3d56f..3b102d8 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloFileOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloFileOutputFormat.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
  * @since 2.0
  */
 public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
+  private static final Class<AccumuloFileOutputFormat> CLASS = AccumuloFileOutputFormat.class;
 
   @Override
   public RecordWriter<Key,Value> getRecordWriter(TaskAttemptContext context) throws IOException {
@@ -93,7 +94,7 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
    * Sets all the information required for this map reduce job.
    */
   public static FileOutputFormatBuilder.PathParams<Job> configure() {
-    return new FileOutputFormatBuilderImpl<>();
+    return new FileOutputFormatBuilderImpl<>(CLASS);
   }
 
 }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java
index 2c6259d..bdb1a08 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloInputFormat.java
@@ -23,8 +23,7 @@ import java.util.Map.Entry;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.accumulo.hadoopImpl.mapreduce.AbstractInputFormat;
-import org.apache.accumulo.hadoopImpl.mapreduce.InputFormatBase;
+import org.apache.accumulo.hadoopImpl.mapreduce.AccumuloRecordReader;
 import org.apache.accumulo.hadoopImpl.mapreduce.InputFormatBuilderImpl;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -80,13 +79,13 @@ public class AccumuloInputFormat extends InputFormat<Key,Value> {
    */
   @Override
   public List<InputSplit> getSplits(JobContext context) throws IOException {
-    return AbstractInputFormat.getSplits(context);
+    return AccumuloRecordReader.getSplits(context, CLASS);
   }
 
   @Override
   public RecordReader<Key,Value> createRecordReader(InputSplit split, TaskAttemptContext context) {
 
-    return new InputFormatBase.RecordReaderBase<Key,Value>() {
+    return new AccumuloRecordReader<Key,Value>(CLASS) {
       @Override
       public boolean nextKeyValue() {
         if (scannerIterator.hasNext()) {
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
index a33e93f..d85b2c8 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
@@ -25,8 +25,9 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.hadoopImpl.mapreduce.AccumuloOutputFormatImpl;
+import org.apache.accumulo.hadoopImpl.mapreduce.AccumuloRecordWriter;
 import org.apache.accumulo.hadoopImpl.mapreduce.OutputFormatBuilderImpl;
+import org.apache.accumulo.hadoopImpl.mapreduce.lib.OutputConfigurator;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -52,10 +53,11 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
  * @since 2.0
  */
 public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
+  private static final Class<AccumuloOutputFormat> CLASS = AccumuloOutputFormat.class;
 
   @Override
   public void checkOutputSpecs(JobContext job) throws IOException {
-    ClientInfo clientInfo = AccumuloOutputFormatImpl.getClientInfo(job);
+    ClientInfo clientInfo = OutputConfigurator.getClientInfo(CLASS, job.getConfiguration());
     String principal = clientInfo.getPrincipal();
     AuthenticationToken token = clientInfo.getAuthenticationToken();
     try (AccumuloClient c = Accumulo.newClient().from(clientInfo.getProperties()).build()) {
@@ -75,7 +77,7 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
   public RecordWriter<Text,Mutation> getRecordWriter(TaskAttemptContext attempt)
       throws IOException {
     try {
-      return new AccumuloOutputFormatImpl.AccumuloRecordWriter(attempt);
+      return new AccumuloRecordWriter(attempt);
     } catch (Exception e) {
       throw new IOException(e);
     }
@@ -85,7 +87,7 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
    * Sets all the information required for this map reduce job.
    */
   public static OutputFormatBuilder.ClientParams<Job> configure() {
-    return new OutputFormatBuilderImpl<>();
+    return new OutputFormatBuilderImpl<>(CLASS);
   }
 
 }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java
index f4ddeb3..1a82679 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloRowInputFormat.java
@@ -24,8 +24,7 @@ import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.accumulo.hadoopImpl.mapreduce.AbstractInputFormat;
-import org.apache.accumulo.hadoopImpl.mapreduce.InputFormatBase;
+import org.apache.accumulo.hadoopImpl.mapreduce.AccumuloRecordReader;
 import org.apache.accumulo.hadoopImpl.mapreduce.InputFormatBuilderImpl;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputFormat;
@@ -62,7 +61,7 @@ public class AccumuloRowInputFormat extends InputFormat<Text,PeekingIterator<Ent
   @Override
   public RecordReader<Text,PeekingIterator<Entry<Key,Value>>> createRecordReader(InputSplit split,
       TaskAttemptContext context) {
-    return new InputFormatBase.RecordReaderBase<Text,PeekingIterator<Entry<Key,Value>>>() {
+    return new AccumuloRecordReader<Text,PeekingIterator<Entry<Key,Value>>>(CLASS) {
       RowIterator rowIterator;
 
       @Override
@@ -97,7 +96,7 @@ public class AccumuloRowInputFormat extends InputFormat<Text,PeekingIterator<Ent
    */
   @Override
   public List<InputSplit> getSplits(JobContext context) throws IOException {
-    return AbstractInputFormat.getSplits(context);
+    return AccumuloRecordReader.getSplits(context, CLASS);
   }
 
   /**
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java
deleted file mode 100644
index 8da2c1a..0000000
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java
+++ /dev/null
@@ -1,568 +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.hadoopImpl.mapred;
-
-import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.security.SecureRandom;
-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 java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-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.ScannerBase;
-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.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.clientImpl.OfflineScanner;
-import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.Table;
-import org.apache.accumulo.core.clientImpl.Tables;
-import org.apache.accumulo.core.clientImpl.TabletLocator;
-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.dataImpl.KeyExtent;
-import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.hadoop.mapred.AccumuloInputFormat;
-import org.apache.accumulo.hadoopImpl.mapreduce.InputTableConfig;
-import org.apache.accumulo.hadoopImpl.mapreduce.SplitUtils;
-import org.apache.accumulo.hadoopImpl.mapreduce.lib.InputConfigurator;
-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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * 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 {
-  protected static final Class<?> CLASS = AccumuloInputFormat.class;
-  private static final Logger log = LoggerFactory.getLogger(CLASS);
-
-  /**
-   * Sets the name of the classloader context on this scanner
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param context
-   *          name of the classloader context
-   * @since 1.8.0
-   */
-  public static void setClassLoaderContext(JobConf job, String context) {
-    InputConfigurator.setClassLoaderContext(CLASS, job, context);
-  }
-
-  /**
-   * Returns the name of the current classloader context set on this scanner
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @return name of the current context
-   * @since 1.8.0
-   */
-  protected static String getClassLoaderContext(JobConf job) {
-    return InputConfigurator.getClassLoaderContext(CLASS, job);
-  }
-
-  /**
-   * Sets connection information needed to communicate with Accumulo for this job
-   *
-   * @param job
-   *          Hadoop job instance to be configured
-   * @param info
-   *          Connection information for Accumulo
-   * @since 2.0.0
-   */
-  public static void setClientInfo(JobConf job, ClientInfo info) {
-    ClientInfo inputInfo = InputConfigurator.updateToken(job.getCredentials(), info);
-    InputConfigurator.setClientInfo(CLASS, job, inputInfo);
-  }
-
-  /**
-   * Set Accumulo client properties file used to connect to Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientPropsFile
-   *          URL to Accumulo client properties file
-   * @since 2.0.0
-   */
-  protected static void setClientPropertiesFile(JobConf job, String clientPropsFile) {
-    InputConfigurator.setClientPropertiesFile(CLASS, job, clientPropsFile);
-  }
-
-  /**
-   * Retrieves {@link ClientInfo} from the configuration
-   *
-   * @param job
-   *          Hadoop job instance configuration
-   * @return {@link ClientInfo} object
-   * @since 2.0.0
-   */
-  protected static ClientInfo getClientInfo(JobConf job) {
-    return InputConfigurator.getClientInfo(CLASS, job);
-  }
-
-  /**
-   * Creates {@link AccumuloClient} from the configuration
-   *
-   * @param job
-   *          Hadoop job instance configuration
-   * @return {@link AccumuloClient} object
-   * @since 2.0.0
-   */
-  protected static AccumuloClient createClient(JobConf job) {
-    return Accumulo.newClient().from(getClientInfo(job).getProperties()).build();
-  }
-
-  /**
-   * Sets the {@link org.apache.accumulo.core.security.Authorizations} used to scan. Must be a
-   * subset of the user's authorization. Defaults to the empty set.
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param auths
-   *          the user's authorizations
-   * @since 1.5.0
-   */
-  public static void setScanAuthorizations(JobConf job, Authorizations auths) {
-    InputConfigurator.setScanAuthorizations(CLASS, job, auths);
-  }
-
-  /**
-   * Gets the authorizations to set for the scans from the configuration.
-   *
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the Accumulo scan authorizations
-   * @since 1.5.0
-   * @see #setScanAuthorizations(JobConf, Authorizations)
-   */
-  protected static Authorizations getScanAuthorizations(JobConf job) {
-    return InputConfigurator.getScanAuthorizations(CLASS, job);
-  }
-
-  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
-  /**
-   * Check whether a configuration is fully configured to be used with an Accumulo
-   * {@link InputFormat}.
-   *
-   * @param job
-   *          the Hadoop context for the configured job
-   * @throws java.io.IOException
-   *           if the context is improperly configured
-   * @since 1.5.0
-   */
-  public static void validateOptions(JobConf job) throws IOException {
-    try (AccumuloClient client = InputConfigurator.createClient(CLASS, job)) {
-      InputConfigurator.validatePermissions(CLASS, job, client);
-    }
-  }
-
-  /**
-   * An abstract base class to be used to create {@link org.apache.hadoop.mapred.RecordReader}
-   * instances that convert from Accumulo
-   * {@link org.apache.accumulo.core.data.Key}/{@link org.apache.accumulo.core.data.Value} pairs to
-   * the user's K/V types.
-   *
-   * Subclasses must implement {@link #next(Object, Object)} to update key and value, and also to
-   * update the following variables:
-   * <ul>
-   * <li>Key {@link #currentKey} (used for progress reporting)</li>
-   * <li>int {@link #numKeysRead} (used for progress reporting)</li>
-   * </ul>
-   */
-  public abstract static class AbstractRecordReader<K,V> implements RecordReader<K,V> {
-    protected long numKeysRead;
-    protected AccumuloClient client;
-    protected Iterator<Map.Entry<Key,Value>> scannerIterator;
-    protected RangeInputSplit split;
-    private org.apache.accumulo.hadoopImpl.mapreduce.RangeInputSplit baseSplit;
-    protected ScannerBase scannerBase;
-
-    /**
-     * Extracts Iterators settings from the context to be used by RecordReader.
-     *
-     * @param job
-     *          the Hadoop job configuration
-     * @return List of iterator settings for given table
-     * @since 1.7.0
-     */
-    protected abstract List<IteratorSetting> jobIterators(JobConf job);
-
-    /**
-     * Configures the iterators on a scanner for the given table name.
-     *
-     * @param job
-     *          the Hadoop job configuration
-     * @param scanner
-     *          the scanner for which to configure the iterators
-     * @since 1.7.0
-     */
-    private void setupIterators(JobConf job, ScannerBase scanner,
-        org.apache.accumulo.hadoopImpl.mapreduce.RangeInputSplit split) {
-      List<IteratorSetting> iterators = null;
-
-      if (split == null) {
-        iterators = jobIterators(job);
-      } else {
-        iterators = split.getIterators();
-        if (iterators == null) {
-          iterators = jobIterators(job);
-        }
-      }
-
-      for (IteratorSetting iterator : iterators)
-        scanner.addScanIterator(iterator);
-    }
-
-    /**
-     * Initialize a scanner over the given input split using this task attempt configuration.
-     */
-    public void initialize(InputSplit inSplit, JobConf job) throws IOException {
-      baseSplit = (org.apache.accumulo.hadoopImpl.mapreduce.RangeInputSplit) inSplit;
-      log.debug("Initializing input split: " + baseSplit);
-
-      client = createClient(job);
-      ClientContext context = (ClientContext) client;
-      Authorizations authorizations = getScanAuthorizations(job);
-      String classLoaderContext = getClassLoaderContext(job);
-      String table = baseSplit.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 = InputConfigurator.getInputTableConfig(CLASS, job,
-          baseSplit.getTableName());
-
-      log.debug("Created client with user: " + context.whoami());
-      log.debug("Creating scanner for table: " + table);
-      log.debug("Authorizations are: " + authorizations);
-
-      if (baseSplit instanceof BatchInputSplit) {
-        BatchScanner scanner;
-        BatchInputSplit multiRangeSplit = (BatchInputSplit) baseSplit;
-
-        try {
-          // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
-          // will not span tablets
-          int scanThreads = 1;
-          scanner = context.createBatchScanner(baseSplit.getTableName(), authorizations,
-              scanThreads);
-          setupIterators(job, scanner, baseSplit);
-          if (classLoaderContext != null) {
-            scanner.setClassLoaderContext(classLoaderContext);
-          }
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
-
-        scanner.setRanges(multiRangeSplit.getRanges());
-        scannerBase = scanner;
-
-      } else if (baseSplit instanceof RangeInputSplit) {
-        split = (RangeInputSplit) baseSplit;
-        Boolean isOffline = baseSplit.isOffline();
-        if (isOffline == null) {
-          isOffline = tableConfig.isOfflineScan();
-        }
-
-        Boolean isIsolated = baseSplit.isIsolatedScan();
-        if (isIsolated == null) {
-          isIsolated = tableConfig.shouldUseIsolatedScanners();
-        }
-
-        Boolean usesLocalIterators = baseSplit.usesLocalIterators();
-        if (usesLocalIterators == null) {
-          usesLocalIterators = tableConfig.shouldUseLocalIterators();
-        }
-
-        Scanner scanner;
-
-        try {
-          if (isOffline) {
-            scanner = new OfflineScanner(context, Table.ID.of(baseSplit.getTableId()),
-                authorizations);
-          } else {
-            scanner = new ScannerImpl(context, Table.ID.of(baseSplit.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(job, scanner, baseSplit);
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
-
-        scanner.setRange(baseSplit.getRange());
-        scannerBase = scanner;
-      } else {
-        throw new IllegalArgumentException("Can not initialize from " + baseSplit.getClass());
-      }
-
-      Collection<IteratorSetting.Column> columns = baseSplit.getFetchedColumns();
-      if (columns == null) {
-        columns = tableConfig.getFetchedColumns();
-      }
-
-      // 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());
-          scannerBase.fetchColumn(c.getFirst(), c.getSecond());
-        } else {
-          log.debug("Fetching column family " + c.getFirst());
-          scannerBase.fetchColumnFamily(c.getFirst());
-        }
-      }
-
-      SamplerConfiguration samplerConfig = baseSplit.getSamplerConfiguration();
-      if (samplerConfig == null) {
-        samplerConfig = tableConfig.getSamplerConfiguration();
-      }
-
-      if (samplerConfig != null) {
-        scannerBase.setSamplerConfiguration(samplerConfig);
-      }
-
-      Map<String,String> executionHints = baseSplit.getExecutionHints();
-      if (executionHints == null || executionHints.size() == 0) {
-        executionHints = tableConfig.getExecutionHints();
-      }
-
-      if (executionHints != null) {
-        scannerBase.setExecutionHints(executionHints);
-      }
-
-      scannerIterator = scannerBase.iterator();
-      numKeysRead = 0;
-    }
-
-    @Override
-    public void close() {
-      if (scannerBase != null) {
-        scannerBase.close();
-      }
-      if (client != null) {
-        client.close();
-      }
-    }
-
-    @Override
-    public long getPos() {
-      return numKeysRead;
-    }
-
-    @Override
-    public float getProgress() {
-      if (numKeysRead > 0 && currentKey == null)
-        return 1.0f;
-      return baseSplit.getProgress(currentKey);
-    }
-
-    protected Key currentKey = null;
-
-  }
-
-  public static Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf job,
-      Table.ID tableId, List<Range> ranges)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    try (AccumuloClient client = createClient(job)) {
-      return InputConfigurator.binOffline(tableId, ranges, (ClientContext) client);
-    }
-  }
-
-  /**
-   * Gets the splits of the tables that have been set on the job by reading the metadata table for
-   * the specified ranges.
-   */
-  public static InputSplit[] getSplits(JobConf job) throws IOException {
-    validateOptions(job);
-
-    Random random = new SecureRandom();
-    LinkedList<InputSplit> splits = new LinkedList<>();
-    Map<String,InputTableConfig> tableConfigs = InputConfigurator.getInputTableConfigs(CLASS, job);
-    try (AccumuloClient client = createClient(job)) {
-      for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
-        String tableName = tableConfigEntry.getKey();
-        InputTableConfig tableConfig = tableConfigEntry.getValue();
-
-        ClientContext context = (ClientContext) client;
-        Table.ID tableId;
-        // resolve table name to id once, and use id from this point forward
-        try {
-          tableId = Tables.getTableId(context, tableName);
-        } catch (TableNotFoundException e) {
-          throw new IOException(e);
-        }
-
-        boolean batchScan = InputConfigurator.isBatchScan(CLASS, job);
-        boolean supportBatchScan = !(tableConfig.isOfflineScan()
-            || tableConfig.shouldUseIsolatedScanners() || tableConfig.shouldUseLocalIterators());
-        if (batchScan && !supportBatchScan)
-          throw new IllegalArgumentException("BatchScanner optimization not available for offline"
-              + " scan, isolated, or local iterators");
-
-        boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
-        if (batchScan && !autoAdjust)
-          throw new IllegalArgumentException(
-              "AutoAdjustRanges must be enabled when using BatchScanner optimization");
-
-        List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges())
-            : tableConfig.getRanges();
-        if (ranges.isEmpty()) {
-          ranges = new ArrayList<>(1);
-          ranges.add(new Range());
-        }
-
-        // get the metadata information for these ranges
-        Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
-        TabletLocator tl;
-        try {
-          if (tableConfig.isOfflineScan()) {
-            binnedRanges = binOfflineTable(job, tableId, ranges);
-            while (binnedRanges == null) {
-              // Some tablets were still online, try again
-              // sleep randomly between 100 and 200 ms
-              sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-              binnedRanges = binOfflineTable(job, tableId, ranges);
-            }
-          } else {
-            tl = InputConfigurator.getTabletLocator(CLASS, job, tableId);
-            // its possible that the cache could contain complete, but old information about a
-            // tables
-            // tablets... so clear it
-            tl.invalidateCache();
-
-            while (!tl.binRanges(context, ranges, binnedRanges).isEmpty()) {
-              String tableIdStr = tableId.canonicalID();
-              if (!Tables.exists(context, tableId))
-                throw new TableDeletedException(tableIdStr);
-              if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
-                throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
-              binnedRanges.clear();
-              log.warn("Unable to locate bins for specified ranges. Retrying.");
-              // sleep randomly between 100 and 200 ms
-              sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-              tl.invalidateCache();
-            }
-          }
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
-
-        HashMap<Range,ArrayList<String>> splitsToAdd = null;
-
-        if (!autoAdjust)
-          splitsToAdd = new HashMap<>();
-
-        HashMap<String,String> hostNameCache = new HashMap<>();
-        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();
-            if (batchScan) {
-              // group ranges by tablet to be read by a BatchScanner
-              ArrayList<Range> clippedRanges = new ArrayList<>();
-              for (Range r : extentRanges.getValue())
-                clippedRanges.add(ke.clip(r));
-
-              BatchInputSplit split = new BatchInputSplit(tableName, tableId, clippedRanges,
-                  new String[] {location});
-              SplitUtils.updateSplit(split, tableConfig);
-
-              splits.add(split);
-            } else {
-              // not grouping by tablet
-              for (Range r : extentRanges.getValue()) {
-                if (autoAdjust) {
-                  // divide ranges into smaller ranges, based on the tablets
-                  RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
-                      ke.clip(r), new String[] {location});
-                  SplitUtils.updateSplit(split, tableConfig);
-                  split.setOffline(tableConfig.isOfflineScan());
-                  split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-                  split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-
-                  splits.add(split);
-                } else {
-                  // don't divide ranges
-                  ArrayList<String> locations = splitsToAdd.get(r);
-                  if (locations == null)
-                    locations = new ArrayList<>(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.canonicalID(),
-                entry.getKey(), entry.getValue().toArray(new String[0]));
-            SplitUtils.updateSplit(split, tableConfig);
-            split.setOffline(tableConfig.isOfflineScan());
-            split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-            split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-
-            splits.add(split);
-          }
-      }
-    }
-
-    return splits.toArray(new InputSplit[splits.size()]);
-  }
-}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloFileOutputFormatImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloFileOutputFormatImpl.java
deleted file mode 100644
index 42ac709..0000000
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloFileOutputFormatImpl.java
+++ /dev/null
@@ -1,135 +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.hadoopImpl.mapred;
-
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.summary.Summarizer;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.hadoop.mapred.AccumuloFileOutputFormat;
-import org.apache.accumulo.hadoopImpl.mapreduce.lib.FileOutputConfigurator;
-import org.apache.hadoop.mapred.JobConf;
-
-public class AccumuloFileOutputFormatImpl {
-
-  private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
-
-  /**
-   * 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(JobConf job, String compressionType) {
-    FileOutputConfigurator.setCompressionType(CLASS, job, 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(JobConf job, long dataBlockSize) {
-    FileOutputConfigurator.setDataBlockSize(CLASS, job, 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(JobConf job, long fileBlockSize) {
-    FileOutputConfigurator.setFileBlockSize(CLASS, job, 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(JobConf job, long indexBlockSize) {
-    FileOutputConfigurator.setIndexBlockSize(CLASS, job, 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(JobConf job, int replication) {
-    FileOutputConfigurator.setReplication(CLASS, job, replication);
-  }
-
-  /**
-   * Specify a sampler to be used when writing out data. This will result in the output file having
-   * sample data.
-   *
-   * @param job
-   *          The Hadoop job instance to be configured
-   * @param samplerConfig
-   *          The configuration for creating sample data in the output file.
-   * @since 1.8.0
-   */
-
-  public static void setSampler(JobConf job, SamplerConfiguration samplerConfig) {
-    FileOutputConfigurator.setSampler(CLASS, job, samplerConfig);
-  }
-
-  /**
-   * Specifies a list of summarizer configurations to create summary data in the output file. Each
-   * Key Value written will be passed to the configured {@link Summarizer}'s.
-   *
-   * @param job
-   *          The Hadoop job instance to be configured
-   * @param summarizerConfigs
-   *          summarizer configurations
-   * @since 2.0.0
-   */
-  public static void setSummarizers(JobConf job, SummarizerConfiguration... summarizerConfigs) {
-    FileOutputConfigurator.setSummarizers(CLASS, job, summarizerConfigs);
-  }
-
-}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java
deleted file mode 100644
index 0823a03..0000000
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java
+++ /dev/null
@@ -1,342 +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.hadoopImpl.mapred;
-
-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.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-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.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.security.SecurityErrorCode;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.TabletId;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.hadoop.mapred.AccumuloOutputFormat;
-import org.apache.accumulo.hadoopImpl.mapreduce.lib.OutputConfigurator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputFormat;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * 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 AccumuloOutputFormatImpl#setClientInfo(JobConf, ClientInfo)}
- * </ul>
- *
- * Other static methods are optional.
- */
-public class AccumuloOutputFormatImpl {
-
-  private static final Class<?> CLASS = AccumuloOutputFormat.class;
-  private static final Logger log = LoggerFactory.getLogger(CLASS);
-
-  /**
-   * Set the connection information needed to communicate with Accumulo in this job.
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param info
-   *          Accumulo connection information
-   * @since 2.0.0
-   */
-  public static void setClientInfo(JobConf job, ClientInfo info) {
-    ClientInfo outInfo = OutputConfigurator.updateToken(job.getCredentials(), info);
-    OutputConfigurator.setClientInfo(CLASS, job, outInfo);
-  }
-
-  /**
-   * Get the connection information needed to communication with Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @since 2.0.0
-   */
-  public static ClientInfo getClientInfo(JobConf job) {
-    return OutputConfigurator.getClientInfo(CLASS, job);
-  }
-
-  /**
-   * 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(JobConf job, String tableName) {
-    OutputConfigurator.setDefaultTableName(CLASS, job, tableName);
-  }
-
-  /**
-   * Gets the default table name from the configuration.
-   *
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the default table name
-   * @since 1.5.0
-   * @see #setDefaultTableName(JobConf, String)
-   */
-  protected static String getDefaultTableName(JobConf job) {
-    return OutputConfigurator.getDefaultTableName(CLASS, job);
-  }
-
-  /**
-   * 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(JobConf job, boolean enableFeature) {
-    OutputConfigurator.setCreateTables(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether tables are permitted to be created as needed.
-   *
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return true if the feature is disabled, false otherwise
-   * @since 1.5.0
-   * @see #setCreateTables(JobConf, boolean)
-   */
-  protected static Boolean canCreateTables(JobConf job) {
-    return OutputConfigurator.canCreateTables(CLASS, job);
-  }
-
-  /**
-   * 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(JobConf job, boolean enableFeature) {
-    OutputConfigurator.setSimulationMode(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether this feature is enabled.
-   *
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setSimulationMode(JobConf, boolean)
-   */
-  protected static Boolean getSimulationMode(JobConf job) {
-    return OutputConfigurator.getSimulationMode(CLASS, job);
-  }
-
-  /**
-   * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
-   */
-  public static class AccumuloRecordWriter implements 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 AccumuloClient client;
-
-    public AccumuloRecordWriter(JobConf job) {
-      this.simulate = getSimulationMode(job);
-      this.createTables = canCreateTables(job);
-
-      if (simulate)
-        log.info("Simulating output only. No writes to tables will occur");
-
-      this.bws = new HashMap<>();
-
-      String tname = getDefaultTableName(job);
-      this.defaultTableName = (tname == null) ? null : new Text(tname);
-
-      if (!simulate) {
-        this.client = Accumulo.newClient().from(getClientInfo(job).getProperties()).build();
-        mtbw = client.createMultiTableBatchWriter();
-      }
-    }
-
-    /**
-     * Push a mutation into a table. If table is null, the defaultTable will be used. If
-     * {@link AccumuloOutputFormatImpl#canCreateTables(JobConf)} 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 (final Exception e) {
-          log.error("Could not add table '" + table + "'", e);
-          throw new IOException(e);
-        }
-
-      try {
-        bws.get(table).addMutation(mutation);
-      } catch (MutationsRejectedException e) {
-        throw new IOException(e);
-      }
-    }
-
-    protected 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 && !client.tableOperations().exists(table)) {
-        try {
-          client.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 | 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(Reporter reporter) throws IOException {
-      log.debug("mutations written: " + mutCount + ", values written: " + valCount);
-      if (simulate)
-        return;
-
-      try {
-        mtbw.close();
-      } catch (MutationsRejectedException e) {
-        if (e.getSecurityErrorCodes().size() >= 0) {
-          HashMap<String,Set<SecurityErrorCode>> tables = new HashMap<>();
-          for (Entry<TabletId,Set<SecurityErrorCode>> ke : e.getSecurityErrorCodes().entrySet()) {
-            String tableId = ke.getKey().getTableId().toString();
-            Set<SecurityErrorCode> secCodes = tables.get(tableId);
-            if (secCodes == null) {
-              secCodes = new HashSet<>();
-              tables.put(tableId, 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());
-        }
-        throw new IOException(e);
-      } finally {
-        client.close();
-      }
-    }
-  }
-
-}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java
new file mode 100644
index 0000000..a2e386e
--- /dev/null
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java
@@ -0,0 +1,451 @@
+/*
+ * 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.hadoopImpl.mapred;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.SecureRandom;
+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 java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.BatchScanner;
+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.ScannerBase;
+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.sample.SamplerConfiguration;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.OfflineScanner;
+import org.apache.accumulo.core.clientImpl.ScannerImpl;
+import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.clientImpl.Tables;
+import org.apache.accumulo.core.clientImpl.TabletLocator;
+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.dataImpl.KeyExtent;
+import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.hadoopImpl.mapreduce.InputTableConfig;
+import org.apache.accumulo.hadoopImpl.mapreduce.SplitUtils;
+import org.apache.accumulo.hadoopImpl.mapreduce.lib.InputConfigurator;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @see org.apache.accumulo.hadoopImpl.mapreduce.AccumuloRecordReader
+ */
+public abstract class AccumuloRecordReader<K,V> implements RecordReader<K,V> {
+  // class to serialize configuration under in the job
+  private final Class<?> CLASS;
+  private static final Logger log = LoggerFactory.getLogger(AccumuloRecordReader.class);
+
+  protected long numKeysRead;
+  protected AccumuloClient client;
+  protected Iterator<Map.Entry<Key,Value>> scannerIterator;
+  protected RangeInputSplit split;
+  private org.apache.accumulo.hadoopImpl.mapreduce.RangeInputSplit baseSplit;
+  protected ScannerBase scannerBase;
+
+  public AccumuloRecordReader(Class<?> callingClass) {
+    this.CLASS = callingClass;
+  }
+
+  /**
+   * Extracts Iterators settings from the context to be used by RecordReader.
+   *
+   * @param job
+   *          the Hadoop job configuration
+   * @return List of iterator settings for given table
+   */
+  private List<IteratorSetting> jobIterators(JobConf job) {
+    return InputConfigurator.getIterators(CLASS, job);
+  }
+
+  /**
+   * Configures the iterators on a scanner for the given table name.
+   *
+   * @param job
+   *          the Hadoop job configuration
+   * @param scanner
+   *          the scanner for which to configure the iterators
+   */
+  private void setupIterators(JobConf job, ScannerBase scanner,
+      org.apache.accumulo.hadoopImpl.mapreduce.RangeInputSplit split) {
+    List<IteratorSetting> iterators = null;
+
+    if (split == null) {
+      iterators = jobIterators(job);
+    } else {
+      iterators = split.getIterators();
+      if (iterators == null) {
+        iterators = jobIterators(job);
+      }
+    }
+
+    for (IteratorSetting iterator : iterators)
+      scanner.addScanIterator(iterator);
+  }
+
+  /**
+   * Initialize a scanner over the given input split using this task attempt configuration.
+   */
+  public void initialize(InputSplit inSplit, JobConf job) throws IOException {
+    baseSplit = (org.apache.accumulo.hadoopImpl.mapreduce.RangeInputSplit) inSplit;
+    log.debug("Initializing input split: " + baseSplit);
+
+    client = createClient(job, CLASS);
+    ClientContext context = (ClientContext) client;
+    Authorizations authorizations = InputConfigurator.getScanAuthorizations(CLASS, job);
+    String classLoaderContext = InputConfigurator.getClassLoaderContext(CLASS, job);
+    String table = baseSplit.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 = InputConfigurator.getInputTableConfig(CLASS, job,
+        baseSplit.getTableName());
+
+    log.debug("Created client with user: " + context.whoami());
+    log.debug("Creating scanner for table: " + table);
+    log.debug("Authorizations are: " + authorizations);
+
+    if (baseSplit instanceof BatchInputSplit) {
+      BatchScanner scanner;
+      BatchInputSplit multiRangeSplit = (BatchInputSplit) baseSplit;
+
+      try {
+        // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
+        // will not span tablets
+        int scanThreads = 1;
+        scanner = context.createBatchScanner(baseSplit.getTableName(), authorizations, scanThreads);
+        setupIterators(job, scanner, baseSplit);
+        if (classLoaderContext != null) {
+          scanner.setClassLoaderContext(classLoaderContext);
+        }
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+
+      scanner.setRanges(multiRangeSplit.getRanges());
+      scannerBase = scanner;
+
+    } else if (baseSplit instanceof RangeInputSplit) {
+      split = (RangeInputSplit) baseSplit;
+      Boolean isOffline = baseSplit.isOffline();
+      if (isOffline == null) {
+        isOffline = tableConfig.isOfflineScan();
+      }
+
+      Boolean isIsolated = baseSplit.isIsolatedScan();
+      if (isIsolated == null) {
+        isIsolated = tableConfig.shouldUseIsolatedScanners();
+      }
+
+      Boolean usesLocalIterators = baseSplit.usesLocalIterators();
+      if (usesLocalIterators == null) {
+        usesLocalIterators = tableConfig.shouldUseLocalIterators();
+      }
+
+      Scanner scanner;
+
+      try {
+        if (isOffline) {
+          scanner = new OfflineScanner(context, Table.ID.of(baseSplit.getTableId()),
+              authorizations);
+        } else {
+          scanner = new ScannerImpl(context, Table.ID.of(baseSplit.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(job, scanner, baseSplit);
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+
+      scanner.setRange(baseSplit.getRange());
+      scannerBase = scanner;
+    } else {
+      throw new IllegalArgumentException("Can not initialize from " + baseSplit.getClass());
+    }
+
+    Collection<IteratorSetting.Column> columns = baseSplit.getFetchedColumns();
+    if (columns == null) {
+      columns = tableConfig.getFetchedColumns();
+    }
+
+    // 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());
+        scannerBase.fetchColumn(c.getFirst(), c.getSecond());
+      } else {
+        log.debug("Fetching column family " + c.getFirst());
+        scannerBase.fetchColumnFamily(c.getFirst());
+      }
+    }
+
+    SamplerConfiguration samplerConfig = baseSplit.getSamplerConfiguration();
+    if (samplerConfig == null) {
+      samplerConfig = tableConfig.getSamplerConfiguration();
+    }
+
+    if (samplerConfig != null) {
+      scannerBase.setSamplerConfiguration(samplerConfig);
+    }
+
+    Map<String,String> executionHints = baseSplit.getExecutionHints();
+    if (executionHints == null || executionHints.size() == 0) {
+      executionHints = tableConfig.getExecutionHints();
+    }
+
+    if (executionHints != null) {
+      scannerBase.setExecutionHints(executionHints);
+    }
+
+    scannerIterator = scannerBase.iterator();
+    numKeysRead = 0;
+  }
+
+  @Override
+  public void close() {
+    if (scannerBase != null) {
+      scannerBase.close();
+    }
+    if (client != null) {
+      client.close();
+    }
+  }
+
+  @Override
+  public long getPos() {
+    return numKeysRead;
+  }
+
+  @Override
+  public float getProgress() {
+    if (numKeysRead > 0 && currentKey == null)
+      return 1.0f;
+    return baseSplit.getProgress(currentKey);
+  }
+
+  protected Key currentKey = null;
+
+  private static Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf job,
+      Table.ID tableId, List<Range> ranges, Class<?> callingClass)
+      throws TableNotFoundException, AccumuloException {
+    try (AccumuloClient client = createClient(job, callingClass)) {
+      return InputConfigurator.binOffline(tableId, ranges, (ClientContext) client);
+    }
+  }
+
+  /**
+   * Gets the splits of the tables that have been set on the job by reading the metadata table for
+   * the specified ranges.
+   */
+  public static InputSplit[] getSplits(JobConf job, Class<?> callingClass) throws IOException {
+    validateOptions(job, callingClass);
+
+    Random random = new SecureRandom();
+    LinkedList<InputSplit> splits = new LinkedList<>();
+    Map<String,InputTableConfig> tableConfigs = InputConfigurator.getInputTableConfigs(callingClass,
+        job);
+    try (AccumuloClient client = createClient(job, callingClass)) {
+      for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
+        String tableName = tableConfigEntry.getKey();
+        InputTableConfig tableConfig = tableConfigEntry.getValue();
+
+        ClientContext context = (ClientContext) client;
+        Table.ID tableId;
+        // resolve table name to id once, and use id from this point forward
+        try {
+          tableId = Tables.getTableId(context, tableName);
+        } catch (TableNotFoundException e) {
+          throw new IOException(e);
+        }
+
+        boolean batchScan = InputConfigurator.isBatchScan(callingClass, job);
+        boolean supportBatchScan = !(tableConfig.isOfflineScan()
+            || tableConfig.shouldUseIsolatedScanners() || tableConfig.shouldUseLocalIterators());
+        if (batchScan && !supportBatchScan)
+          throw new IllegalArgumentException("BatchScanner optimization not available for offline"
+              + " scan, isolated, or local iterators");
+
+        boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
+        if (batchScan && !autoAdjust)
+          throw new IllegalArgumentException(
+              "AutoAdjustRanges must be enabled when using BatchScanner optimization");
+
+        List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges())
+            : tableConfig.getRanges();
+        if (ranges.isEmpty()) {
+          ranges = new ArrayList<>(1);
+          ranges.add(new Range());
+        }
+
+        // get the metadata information for these ranges
+        Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
+        TabletLocator tl;
+        try {
+          if (tableConfig.isOfflineScan()) {
+            binnedRanges = binOfflineTable(job, tableId, ranges, callingClass);
+            while (binnedRanges == null) {
+              // Some tablets were still online, try again
+              // sleep randomly between 100 and 200 ms
+              sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
+              binnedRanges = binOfflineTable(job, tableId, ranges, callingClass);
+            }
+          } else {
+            tl = InputConfigurator.getTabletLocator(callingClass, job, tableId);
+            // its possible that the cache could contain complete, but old information about a
+            // tables
+            // tablets... so clear it
+            tl.invalidateCache();
+
+            while (!tl.binRanges(context, ranges, binnedRanges).isEmpty()) {
+              String tableIdStr = tableId.canonicalID();
+              if (!Tables.exists(context, tableId))
+                throw new TableDeletedException(tableIdStr);
+              if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
+                throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
+              binnedRanges.clear();
+              log.warn("Unable to locate bins for specified ranges. Retrying.");
+              // sleep randomly between 100 and 200 ms
+              sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
+              tl.invalidateCache();
+            }
+          }
+        } catch (Exception e) {
+          throw new IOException(e);
+        }
+
+        HashMap<Range,ArrayList<String>> splitsToAdd = null;
+
+        if (!autoAdjust)
+          splitsToAdd = new HashMap<>();
+
+        HashMap<String,String> hostNameCache = new HashMap<>();
+        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();
+            if (batchScan) {
+              // group ranges by tablet to be read by a BatchScanner
+              ArrayList<Range> clippedRanges = new ArrayList<>();
+              for (Range r : extentRanges.getValue())
+                clippedRanges.add(ke.clip(r));
+
+              BatchInputSplit split = new BatchInputSplit(tableName, tableId, clippedRanges,
+                  new String[] {location});
+              SplitUtils.updateSplit(split, tableConfig);
+
+              splits.add(split);
+            } else {
+              // not grouping by tablet
+              for (Range r : extentRanges.getValue()) {
+                if (autoAdjust) {
+                  // divide ranges into smaller ranges, based on the tablets
+                  RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+                      ke.clip(r), new String[] {location});
+                  SplitUtils.updateSplit(split, tableConfig);
+                  split.setOffline(tableConfig.isOfflineScan());
+                  split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
+                  split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
+
+                  splits.add(split);
+                } else {
+                  // don't divide ranges
+                  ArrayList<String> locations = splitsToAdd.get(r);
+                  if (locations == null)
+                    locations = new ArrayList<>(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.canonicalID(),
+                entry.getKey(), entry.getValue().toArray(new String[0]));
+            SplitUtils.updateSplit(split, tableConfig);
+            split.setOffline(tableConfig.isOfflineScan());
+            split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
+            split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
+
+            splits.add(split);
+          }
+      }
+    }
+
+    return splits.toArray(new InputSplit[splits.size()]);
+  }
+
+  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
+  /**
+   * Check whether a configuration is fully configured to be used with an Accumulo
+   * {@link InputFormat}.
+   */
+  private static void validateOptions(JobConf job, Class<?> callingClass) throws IOException {
+    try (AccumuloClient client = InputConfigurator.createClient(callingClass, job)) {
+      InputConfigurator.validatePermissions(callingClass, job, client);
+    }
+  }
+
+  /**
+   * Creates {@link AccumuloClient} from the configuration
+   */
+  private static AccumuloClient createClient(JobConf job, Class<?> callingClass) {
+    return Accumulo.newClient()
+        .from(InputConfigurator.getClientInfo(callingClass, job).getProperties()).build();
+  }
+}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordWriter.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordWriter.java
new file mode 100644
index 0000000..4ff1323
--- /dev/null
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordWriter.java
@@ -0,0 +1,212 @@
+/*
+ * 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.hadoopImpl.mapred;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+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.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.security.SecurityErrorCode;
+import org.apache.accumulo.core.data.ColumnUpdate;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.hadoop.mapred.AccumuloOutputFormat;
+import org.apache.accumulo.hadoopImpl.mapreduce.lib.OutputConfigurator;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.mapred.Reporter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
+ */
+public class AccumuloRecordWriter implements RecordWriter<Text,Mutation> {
+  // class to serialize configuration under in the job
+  private static final Class<AccumuloOutputFormat> CLASS = AccumuloOutputFormat.class;
+  private static final Logger log = LoggerFactory.getLogger(AccumuloRecordWriter.class);
+  private MultiTableBatchWriter mtbw = null;
+  private HashMap<Text,BatchWriter> bws;
+  private Text defaultTableName;
+
+  private boolean simulate;
+  private boolean createTables;
+
+  private long mutCount = 0;
+  private long valCount = 0;
+
+  private AccumuloClient client;
+
+  public AccumuloRecordWriter(JobConf job) {
+    this.simulate = OutputConfigurator.getSimulationMode(CLASS, job);
+    this.createTables = OutputConfigurator.canCreateTables(CLASS, job);
+
+    if (simulate)
+      log.info("Simulating output only. No writes to tables will occur");
+
+    this.bws = new HashMap<>();
+
+    String tname = OutputConfigurator.getDefaultTableName(CLASS, job);
+    this.defaultTableName = (tname == null) ? null : new Text(tname);
+
+    if (!simulate) {
+      this.client = Accumulo.newClient()
+          .from(OutputConfigurator.getClientInfo(CLASS, job).getProperties()).build();
+      mtbw = client.createMultiTableBatchWriter();
+    }
+  }
+
+  /**
+   * Push a mutation into a table. If table is null, the defaultTable will be used. If
+   * OutputFormatBuilder#createTables() 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 (final Exception e) {
+        log.error("Could not add table '" + table + "'", e);
+        throw new IOException(e);
+      }
+
+    try {
+      bws.get(table).addMutation(mutation);
+    } catch (MutationsRejectedException e) {
+      throw new IOException(e);
+    }
+  }
+
+  protected 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 && !client.tableOperations().exists(table)) {
+      try {
+        client.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 | 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(Reporter reporter) throws IOException {
+    log.debug("mutations written: " + mutCount + ", values written: " + valCount);
+    if (simulate)
+      return;
+
+    try {
+      mtbw.close();
+    } catch (MutationsRejectedException e) {
+      if (e.getSecurityErrorCodes().size() >= 0) {
+        HashMap<String,Set<SecurityErrorCode>> tables = new HashMap<>();
+        for (Map.Entry<TabletId,Set<SecurityErrorCode>> ke : e.getSecurityErrorCodes().entrySet()) {
+          String tableId = ke.getKey().getTableId().toString();
+          Set<SecurityErrorCode> secCodes = tables.get(tableId);
+          if (secCodes == null) {
+            secCodes = new HashSet<>();
+            tables.put(tableId, 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());
+      }
+      throw new IOException(e);
+    } finally {
+      client.close();
+    }
+  }
+}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/InputFormatBase.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/InputFormatBase.java
deleted file mode 100644
index acfa559..0000000
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/InputFormatBase.java
+++ /dev/null
@@ -1,333 +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.hadoopImpl.mapred;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-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.ScannerBase;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.hadoopImpl.mapreduce.lib.InputConfigurator;
-import org.apache.hadoop.mapred.JobConf;
-
-public abstract class InputFormatBase extends AbstractInputFormat {
-
-  /**
-   * 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
-   * @see TableOperations#splitRangeByTablets(String, Range, int)
-   * @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.
-   */
-  public static void fetchColumns(JobConf job,
-      Collection<IteratorSetting.Column> columnFamilyColumnQualifierPairs) {
-    InputConfigurator.fetchColumns(CLASS, job, columnFamilyColumnQualifierPairs);
-  }
-
-  /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this
-   * configuration.
-   */
-  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. Must be enabled when
-   * {@link #setBatchScan(JobConf, boolean)} is true.
-   *
-   * @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);
-  }
-
-  /**
-   * 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);
-  }
-
-  /**
-   * Controls the use of the {@link org.apache.accumulo.core.client.BatchScanner} in this job. Using
-   * this feature will group Ranges by their source tablet, producing an InputSplit per tablet
-   * rather than per Range. This batching helps to reduce overhead when querying a large number of
-   * small ranges. (ex: when doing quad-tree decomposition for spatial queries)
-   * <p>
-   * In order to achieve good locality of InputSplits this option always clips the input Ranges to
-   * tablet boundaries. This may result in one input Range contributing to several InputSplits.
-   * <p>
-   * Note: that the value of {@link #setAutoAdjustRanges(JobConf, boolean)} is ignored and is
-   * assumed to be true when BatchScan option is enabled.
-   * <p>
-   * This configuration is incompatible with:
-   * <ul>
-   * <li>{@link #setOfflineTableScan(JobConf, boolean)}</li>
-   * <li>{@link #setLocalIterators(JobConf, boolean)}</li>
-   * <li>{@link #setScanIsolation(JobConf, boolean)}</li>
-   * </ul>
-   * <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.7.0
-   */
-  public static void setBatchScan(JobConf job, boolean enableFeature) {
-    InputConfigurator.setBatchScan(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the {@link org.apache.accumulo.core.client.BatchScanner}
-   * feature enabled.
-   *
-   * @param job
-   *          the Hadoop context for the configured job
-   * @since 1.7.0
-   * @see #setBatchScan(JobConf, boolean)
-   */
-  protected static boolean isBatchScan(JobConf job) {
-    return InputConfigurator.isBatchScan(CLASS, job);
-  }
-
-  /**
-   * Causes input format to read sample data. If sample data was created using a different
-   * configuration or a tables sampler configuration changes while reading data, then the input
-   * format will throw an error.
-   *
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param samplerConfig
-   *          The sampler configuration that sample must have been created with inorder for reading
-   *          sample data to succeed.
-   *
-   * @since 1.8.0
-   * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
-   */
-  public static void setSamplerConfiguration(JobConf job, SamplerConfiguration samplerConfig) {
-    InputConfigurator.setSamplerConfiguration(CLASS, job, samplerConfig);
-  }
-
-  /**
-   * Set these execution hints on scanners created for input splits. See
-   * {@link ScannerBase#setExecutionHints(java.util.Map)}
-   *
-   * @since 2.0.0
-   */
-  public static void setExecutionHints(JobConf job, Map<String,String> hints) {
-    InputConfigurator.setExecutionHints(CLASS, job, hints);
-  }
-
-  public abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
-
-    @Override
-    protected List<IteratorSetting> jobIterators(JobConf job) {
-      return getIterators(job);
-    }
-  }
-}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java
deleted file mode 100644
index cdd929f..0000000
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java
+++ /dev/null
@@ -1,587 +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.hadoopImpl.mapreduce;
-
-import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.security.SecureRandom;
-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 java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-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.ScannerBase;
-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.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.clientImpl.OfflineScanner;
-import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.Table;
-import org.apache.accumulo.core.clientImpl.Tables;
-import org.apache.accumulo.core.clientImpl.TabletLocator;
-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.dataImpl.KeyExtent;
-import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.hadoopImpl.mapreduce.lib.InputConfigurator;
-import org.apache.hadoop.io.Text;
-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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * 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 {
-
-  protected static final Class<?> CLASS = AccumuloInputFormat.class;
-  private static final Logger log = LoggerFactory.getLogger(CLASS);
-
-  /**
-   * Sets the name of the classloader context on this scanner
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param context
-   *          name of the classloader context
-   * @since 1.8.0
-   */
-  public static void setClassLoaderContext(Job job, String context) {
-    InputConfigurator.setClassLoaderContext(CLASS, job.getConfiguration(), context);
-  }
-
-  /**
-   * Returns the name of the current classloader context set on this scanner
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @return name of the current context
-   * @since 1.8.0
-   */
-  protected static String getClassLoaderContext(JobContext job) {
-    return InputConfigurator.getClassLoaderContext(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Sets connection information needed to communicate with Accumulo for this job
-   *
-   * @param job
-   *          Hadoop job instance to be configured
-   * @param info
-   *          Connection information for Accumulo
-   * @since 2.0.0
-   */
-  public static void setClientInfo(Job job, ClientInfo info) {
-    ClientInfo inputInfo = InputConfigurator.updateToken(job.getCredentials(), info);
-    InputConfigurator.setClientInfo(CLASS, job.getConfiguration(), inputInfo);
-  }
-
-  /**
-   * Set Accumulo client properties file used to connect to Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientPropsFile
-   *          URL (hdfs:// or http://) to Accumulo client properties file
-   * @since 2.0.0
-   */
-  protected static void setClientPropertiesFile(Job job, String clientPropsFile) {
-    InputConfigurator.setClientPropertiesFile(CLASS, job.getConfiguration(), clientPropsFile);
-  }
-
-  /**
-   * Gets the {@link ClientInfo} from the configuration
-   *
-   * @param context
-   *          Hadoop job context
-   * @return ClientInfo
-   * @since 2.0.0
-   */
-  public static ClientInfo getClientInfo(JobContext context) {
-    return InputConfigurator.getClientInfo(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Creates {@link AccumuloClient} from the configuration
-   *
-   * @param context
-   *          Hadoop job context
-   * @return AccumuloClient
-   * @since 2.0.0
-   */
-  public static AccumuloClient createClient(JobContext context) {
-    return Accumulo.newClient().from(getClientInfo(context).getProperties()).build();
-  }
-
-  /**
-   * 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, context.getConfiguration());
-  }
-
-  // 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
-   */
-  public static void validateOptions(JobContext context) throws IOException {
-    try (
-        AccumuloClient client = InputConfigurator.createClient(CLASS, context.getConfiguration())) {
-      InputConfigurator.validatePermissions(CLASS, context.getConfiguration(), client);
-    }
-  }
-
-  /**
-   * 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 AccumuloClient client;
-    protected Iterator<Map.Entry<Key,Value>> scannerIterator;
-    protected ScannerBase scannerBase;
-    protected RangeInputSplit split;
-
-    /**
-     * Extracts Iterators settings from the context to be used by RecordReader.
-     *
-     * @param context
-     *          the Hadoop context for the configured job
-     * @return List of iterator settings for given table
-     * @since 1.7.0
-     */
-    protected abstract List<IteratorSetting> contextIterators(TaskAttemptContext context);
-
-    /**
-     * Configures the iterators on a scanner for the given table name. Will attempt to use
-     * configuration from the InputSplit, on failure will try to extract them from
-     * TaskAttemptContext.
-     *
-     * @param context
-     *          the Hadoop context for the configured job
-     * @param scanner
-     *          the scanner for which to configure the iterators
-     * @param split
-     *          InputSplit containing configurations
-     * @since 1.7.0
-     */
-    private void setupIterators(TaskAttemptContext context, ScannerBase scanner,
-        RangeInputSplit split) {
-      List<IteratorSetting> iterators = null;
-
-      if (split == null) {
-        iterators = contextIterators(context);
-      } else {
-        iterators = split.getIterators();
-        if (iterators == null) {
-          iterators = contextIterators(context);
-        }
-      }
-
-      for (IteratorSetting iterator : iterators)
-        scanner.addScanIterator(iterator);
-    }
-
-    @Override
-    public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
-
-      split = (RangeInputSplit) inSplit;
-      log.debug("Initializing input split: " + split);
-
-      client = createClient(attempt);
-      ClientContext context = (ClientContext) client;
-      Authorizations authorizations = getScanAuthorizations(attempt);
-      String classLoaderContext = getClassLoaderContext(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 = InputConfigurator.getInputTableConfig(CLASS,
-          attempt.getConfiguration(), split.getTableName());
-
-      log.debug("Creating client with user: " + client.whoami());
-      log.debug("Creating scanner for table: " + table);
-      log.debug("Authorizations are: " + authorizations);
-
-      if (split instanceof BatchInputSplit) {
-        BatchInputSplit batchSplit = (BatchInputSplit) split;
-
-        BatchScanner scanner;
-        try {
-          // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
-          // will not span tablets
-          int scanThreads = 1;
-          scanner = context.createBatchScanner(split.getTableName(), authorizations, scanThreads);
-          setupIterators(attempt, scanner, split);
-          if (classLoaderContext != null) {
-            scanner.setClassLoaderContext(classLoaderContext);
-          }
-        } catch (Exception e) {
-          e.printStackTrace();
-          throw new IOException(e);
-        }
-
-        scanner.setRanges(batchSplit.getRanges());
-        scannerBase = scanner;
-      } else {
-        Scanner scanner;
-
-        Boolean isOffline = split.isOffline();
-        if (isOffline == null) {
-          isOffline = tableConfig.isOfflineScan();
-        }
-
-        Boolean isIsolated = split.isIsolatedScan();
-        if (isIsolated == null) {
-          isIsolated = tableConfig.shouldUseIsolatedScanners();
-        }
-
-        Boolean usesLocalIterators = split.usesLocalIterators();
-        if (usesLocalIterators == null) {
-          usesLocalIterators = tableConfig.shouldUseLocalIterators();
-        }
-
-        try {
-          if (isOffline) {
-            scanner = new OfflineScanner(context, Table.ID.of(split.getTableId()), authorizations);
-          } else {
-            // Not using public API to create scanner so that we can use table ID
-            // Table ID is used in case of renames during M/R job
-            scanner = new ScannerImpl(context, Table.ID.of(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);
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
-
-        scanner.setRange(split.getRange());
-        scannerBase = scanner;
-
-      }
-
-      Collection<IteratorSetting.Column> columns = split.getFetchedColumns();
-      if (columns == null) {
-        columns = tableConfig.getFetchedColumns();
-      }
-
-      // 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());
-          scannerBase.fetchColumn(c.getFirst(), c.getSecond());
-        } else {
-          log.debug("Fetching column family " + c.getFirst());
-          scannerBase.fetchColumnFamily(c.getFirst());
-        }
-      }
-
-      SamplerConfiguration samplerConfig = split.getSamplerConfiguration();
-      if (samplerConfig == null) {
-        samplerConfig = tableConfig.getSamplerConfiguration();
-      }
-
-      if (samplerConfig != null) {
-        scannerBase.setSamplerConfiguration(samplerConfig);
-      }
-
-      Map<String,String> executionHints = split.getExecutionHints();
-      if (executionHints == null || executionHints.isEmpty()) {
-        executionHints = tableConfig.getExecutionHints();
-      }
-
-      if (executionHints != null) {
-        scannerBase.setExecutionHints(executionHints);
-      }
-
-      scannerIterator = scannerBase.iterator();
-      numKeysRead = 0;
-    }
-
-    @Override
-    public void close() {
-      if (scannerBase != null) {
-        scannerBase.close();
-      }
-      if (client != null) {
-        client.close();
-      }
-    }
-
-    @Override
-    public float getProgress() {
-      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() {
-      return currentK;
-    }
-
-    @Override
-    public V getCurrentValue() {
-      return currentV;
-    }
-  }
-
-  public static Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext context,
-      Table.ID tableId, List<Range> ranges)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    try (AccumuloClient client = createClient(context)) {
-      return InputConfigurator.binOffline(tableId, ranges, (ClientContext) client);
-    }
-  }
-
-  public static List<InputSplit> getSplits(JobContext context) throws IOException {
-    validateOptions(context);
-    Random random = new SecureRandom();
-    LinkedList<InputSplit> splits = new LinkedList<>();
-    try (AccumuloClient client = createClient(context)) {
-      Map<String,InputTableConfig> tableConfigs = InputConfigurator.getInputTableConfigs(CLASS,
-          context.getConfiguration());
-      for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
-
-        String tableName = tableConfigEntry.getKey();
-        InputTableConfig tableConfig = tableConfigEntry.getValue();
-
-        ClientContext clientContext = (ClientContext) client;
-        Table.ID tableId;
-        // resolve table name to id once, and use id from this point forward
-        try {
-          tableId = Tables.getTableId(clientContext, tableName);
-        } catch (TableNotFoundException e) {
-          throw new IOException(e);
-        }
-
-        boolean batchScan = InputConfigurator.isBatchScan(CLASS, context.getConfiguration());
-        boolean supportBatchScan = !(tableConfig.isOfflineScan()
-            || tableConfig.shouldUseIsolatedScanners() || tableConfig.shouldUseLocalIterators());
-        if (batchScan && !supportBatchScan)
-          throw new IllegalArgumentException("BatchScanner optimization not available for offline"
-              + " scan, isolated, or local iterators");
-
-        boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
-        if (batchScan && !autoAdjust)
-          throw new IllegalArgumentException(
-              "AutoAdjustRanges must be enabled when using BatchScanner optimization");
-
-        List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges())
-            : tableConfig.getRanges();
-        if (ranges.isEmpty()) {
-          ranges = new ArrayList<>(1);
-          ranges.add(new Range());
-        }
-
-        // get the metadata information for these ranges
-        Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
-        TabletLocator tl;
-        try {
-          if (tableConfig.isOfflineScan()) {
-            binnedRanges = binOfflineTable(context, tableId, ranges);
-            while (binnedRanges == null) {
-              // Some tablets were still online, try again
-              // sleep randomly between 100 and 200 ms
-              sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-              binnedRanges = binOfflineTable(context, tableId, ranges);
-
-            }
-          } else {
-            tl = InputConfigurator.getTabletLocator(CLASS, context.getConfiguration(), tableId);
-            // its possible that the cache could contain complete, but old information about a
-            // tables tablets... so clear it
-            tl.invalidateCache();
-
-            while (!tl.binRanges(clientContext, ranges, binnedRanges).isEmpty()) {
-              String tableIdStr = tableId.canonicalID();
-              if (!Tables.exists(clientContext, tableId))
-                throw new TableDeletedException(tableIdStr);
-              if (Tables.getTableState(clientContext, tableId) == TableState.OFFLINE)
-                throw new TableOfflineException(Tables.getTableOfflineMsg(clientContext, tableId));
-              binnedRanges.clear();
-              log.warn("Unable to locate bins for specified ranges. Retrying.");
-              // sleep randomly between 100 and 200 ms
-              sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-              tl.invalidateCache();
-            }
-          }
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
-
-        // all of this code will add either range per each locations or split ranges and add
-        // range-location split
-        // Map from Range to Array of Locations, we only use this if we're don't split
-        HashMap<Range,ArrayList<String>> splitsToAdd = null;
-
-        if (!autoAdjust)
-          splitsToAdd = new HashMap<>();
-
-        HashMap<String,String> hostNameCache = new HashMap<>();
-        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();
-            if (batchScan) {
-              // group ranges by tablet to be read by a BatchScanner
-              ArrayList<Range> clippedRanges = new ArrayList<>();
-              for (Range r : extentRanges.getValue())
-                clippedRanges.add(ke.clip(r));
-              BatchInputSplit split = new BatchInputSplit(tableName, tableId, clippedRanges,
-                  new String[] {location});
-              SplitUtils.updateSplit(split, tableConfig);
-
-              splits.add(split);
-            } else {
-              // not grouping by tablet
-              for (Range r : extentRanges.getValue()) {
-                if (autoAdjust) {
-                  // divide ranges into smaller ranges, based on the tablets
-                  RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
-                      ke.clip(r), new String[] {location});
-                  SplitUtils.updateSplit(split, tableConfig);
-                  split.setOffline(tableConfig.isOfflineScan());
-                  split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-                  split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-                  splits.add(split);
-                } else {
-                  // don't divide ranges
-                  ArrayList<String> locations = splitsToAdd.get(r);
-                  if (locations == null)
-                    locations = new ArrayList<>(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.canonicalID(),
-                entry.getKey(), entry.getValue().toArray(new String[0]));
-            SplitUtils.updateSplit(split, tableConfig);
-            split.setOffline(tableConfig.isOfflineScan());
-            split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-            split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-
-            splits.add(split);
-          }
-      }
-    }
-    return splits;
-  }
-
-}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloFileOutputFormatImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloFileOutputFormatImpl.java
deleted file mode 100644
index a163de4..0000000
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloFileOutputFormatImpl.java
+++ /dev/null
@@ -1,134 +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.hadoopImpl.mapreduce;
-
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.summary.Summarizer;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.hadoop.mapreduce.AccumuloFileOutputFormat;
-import org.apache.accumulo.hadoopImpl.mapreduce.lib.FileOutputConfigurator;
-import org.apache.hadoop.mapreduce.Job;
-
-public class AccumuloFileOutputFormatImpl {
-
-  private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
-
-  /**
-   * 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);
-  }
-
-  /**
-   * Specify a sampler to be used when writing out data. This will result in the output file having
-   * sample data.
-   *
-   * @param job
-   *          The Hadoop job instance to be configured
-   * @param samplerConfig
-   *          The configuration for creating sample data in the output file.
-   * @since 1.8.0
-   */
-
-  public static void setSampler(Job job, SamplerConfiguration samplerConfig) {
-    FileOutputConfigurator.setSampler(CLASS, job.getConfiguration(), samplerConfig);
-  }
-
-  /**
-   * Specifies a list of summarizer configurations to create summary data in the output file. Each
-   * Key Value written will be passed to the configured {@link Summarizer}'s.
-   *
-   * @param job
-   *          The Hadoop job instance to be configured
-   * @param summarizerConfigs
-   *          summarizer configurations
-   * @since 2.0.0
-   */
-  public static void setSummarizers(Job job, SummarizerConfiguration... summarizerConfigs) {
-    FileOutputConfigurator.setSummarizers(CLASS, job.getConfiguration(), summarizerConfigs);
-  }
-}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java
deleted file mode 100644
index 8750d74..0000000
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java
+++ /dev/null
@@ -1,344 +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.hadoopImpl.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.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-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.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.security.SecurityErrorCode;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.TabletId;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat;
-import org.apache.accumulo.hadoopImpl.mapreduce.lib.OutputConfigurator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * 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 AccumuloOutputFormatImpl#setClientInfo(Job, ClientInfo)}
- * </ul>
- *
- * Other static methods are optional.
- */
-public class AccumuloOutputFormatImpl {
-
-  private static final Class<?> CLASS = AccumuloOutputFormat.class;
-  private static final Logger log = LoggerFactory.getLogger(CLASS);
-
-  /**
-   * Set the connection information needed to communicate with Accumulo in this job.
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param info
-   *          Accumulo connection information
-   * @since 2.0.0
-   */
-  public static void setClientInfo(Job job, ClientInfo info) {
-    OutputConfigurator.setClientInfo(CLASS, job.getConfiguration(), info);
-  }
-
-  /**
-   * Get connection information from this job
-   *
-   * @param context
-   *          Hadoop job context
-   * @return {@link ClientInfo}
-   *
-   * @since 2.0.0
-   */
-  public static ClientInfo getClientInfo(JobContext context) {
-    return OutputConfigurator.getClientInfo(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * 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, context.getConfiguration());
-  }
-
-  /**
-   * 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, context.getConfiguration());
-  }
-
-  /**
-   * 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, context.getConfiguration());
-  }
-
-  /**
-   * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
-   */
-  public 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 AccumuloClient client;
-
-    public AccumuloRecordWriter(TaskAttemptContext 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<>();
-
-      String tname = getDefaultTableName(context);
-      this.defaultTableName = (tname == null) ? null : new Text(tname);
-
-      if (!simulate) {
-        this.client = Accumulo.newClient().from(getClientInfo(context).getProperties()).build();
-        mtbw = client.createMultiTableBatchWriter();
-      }
-    }
-
-    /**
-     * Push a mutation into a table. If table is null, the defaultTable will be used. If
-     * {@link AccumuloOutputFormatImpl#canCreateTables(JobContext)} 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) {
-          log.error("Could not add table '" + table + "'", e);
-          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 && !client.tableOperations().exists(table)) {
-        try {
-          client.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 | 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 {
-      log.debug("mutations written: " + mutCount + ", values written: " + valCount);
-      if (simulate)
-        return;
-
-      try {
-        mtbw.close();
-      } catch (MutationsRejectedException e) {
-        if (e.getSecurityErrorCodes().size() >= 0) {
-          HashMap<String,Set<SecurityErrorCode>> tables = new HashMap<>();
-          for (Entry<TabletId,Set<SecurityErrorCode>> ke : e.getSecurityErrorCodes().entrySet()) {
-            String tableId = ke.getKey().getTableId().toString();
-            Set<SecurityErrorCode> secCodes = tables.get(tableId);
-            if (secCodes == null) {
-              secCodes = new HashSet<>();
-              tables.put(tableId, 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());
-        }
-        throw new IOException(e);
-      } finally {
-        client.close();
-      }
-    }
-  }
-
-}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java
new file mode 100644
index 0000000..32fc759
--- /dev/null
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java
@@ -0,0 +1,479 @@
+/*
+ * 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.hadoopImpl.mapreduce;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.SecureRandom;
+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 java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.BatchScanner;
+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.ScannerBase;
+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.sample.SamplerConfiguration;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
+import org.apache.accumulo.core.clientImpl.OfflineScanner;
+import org.apache.accumulo.core.clientImpl.ScannerImpl;
+import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.clientImpl.Tables;
+import org.apache.accumulo.core.clientImpl.TabletLocator;
+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.dataImpl.KeyExtent;
+import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.hadoopImpl.mapreduce.lib.InputConfigurator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An implementation of {@link org.apache.hadoop.mapreduce.RecordReader} that converts Accumulo
+ * {@link org.apache.accumulo.core.data.Key}/{@link org.apache.accumulo.core.data.Value} pairs to
+ * the user's K/V types.
+ */
+public abstract class AccumuloRecordReader<K,V> extends RecordReader<K,V> {
+  private static final Logger log = LoggerFactory.getLogger(AccumuloRecordReader.class);
+  // class to serialize configuration under in the job
+  private final Class<?> CLASS;
+  protected long numKeysRead;
+  protected AccumuloClient client;
+  protected Iterator<Map.Entry<Key,Value>> scannerIterator;
+  protected ScannerBase scannerBase;
+  protected RangeInputSplit split;
+
+  public AccumuloRecordReader(Class<?> callingClass) {
+    this.CLASS = callingClass;
+  }
+
+  /**
+   * 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;
+
+  /**
+   * Extracts Iterators settings from the context to be used by RecordReader.
+   *
+   * @param context
+   *          the Hadoop context for the configured job
+   * @return List of iterator settings for given table
+   */
+  private List<IteratorSetting> contextIterators(TaskAttemptContext context) {
+    return InputConfigurator.getIterators(CLASS, context.getConfiguration());
+  }
+
+  /**
+   * Configures the iterators on a scanner for the given table name. Will attempt to use
+   * configuration from the InputSplit, on failure will try to extract them from TaskAttemptContext.
+   *
+   * @param context
+   *          the Hadoop context for the configured job
+   * @param scanner
+   *          the scanner for which to configure the iterators
+   * @param split
+   *          InputSplit containing configurations
+   */
+  private void setupIterators(TaskAttemptContext context, ScannerBase scanner,
+      RangeInputSplit split) {
+    List<IteratorSetting> iterators = null;
+
+    if (split == null) {
+      iterators = contextIterators(context);
+    } else {
+      iterators = split.getIterators();
+      if (iterators == null) {
+        iterators = contextIterators(context);
+      }
+    }
+
+    for (IteratorSetting iterator : iterators)
+      scanner.addScanIterator(iterator);
+  }
+
+  @Override
+  public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
+
+    split = (RangeInputSplit) inSplit;
+    log.debug("Initializing input split: " + split);
+    Configuration conf = attempt.getConfiguration();
+
+    client = createClient(attempt, this.CLASS);
+    ClientContext context = (ClientContext) client;
+    Authorizations authorizations = InputConfigurator.getScanAuthorizations(CLASS, conf);
+    String classLoaderContext = InputConfigurator.getClassLoaderContext(CLASS, conf);
+    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 = InputConfigurator.getInputTableConfig(CLASS, conf,
+        split.getTableName());
+
+    log.debug("Creating client with user: " + client.whoami());
+    log.debug("Creating scanner for table: " + table);
+    log.debug("Authorizations are: " + authorizations);
+
+    if (split instanceof BatchInputSplit) {
+      BatchInputSplit batchSplit = (BatchInputSplit) split;
+
+      BatchScanner scanner;
+      try {
+        // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
+        // will not span tablets
+        int scanThreads = 1;
+        scanner = context.createBatchScanner(split.getTableName(), authorizations, scanThreads);
+        setupIterators(attempt, scanner, split);
+        if (classLoaderContext != null) {
+          scanner.setClassLoaderContext(classLoaderContext);
+        }
+      } catch (Exception e) {
+        e.printStackTrace();
+        throw new IOException(e);
+      }
+
+      scanner.setRanges(batchSplit.getRanges());
+      scannerBase = scanner;
+    } else {
+      Scanner scanner;
+
+      Boolean isOffline = split.isOffline();
+      if (isOffline == null) {
+        isOffline = tableConfig.isOfflineScan();
+      }
+
+      Boolean isIsolated = split.isIsolatedScan();
+      if (isIsolated == null) {
+        isIsolated = tableConfig.shouldUseIsolatedScanners();
+      }
+
+      Boolean usesLocalIterators = split.usesLocalIterators();
+      if (usesLocalIterators == null) {
+        usesLocalIterators = tableConfig.shouldUseLocalIterators();
+      }
+
+      try {
+        if (isOffline) {
+          scanner = new OfflineScanner(context, Table.ID.of(split.getTableId()), authorizations);
+        } else {
+          // Not using public API to create scanner so that we can use table ID
+          // Table ID is used in case of renames during M/R job
+          scanner = new ScannerImpl(context, Table.ID.of(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);
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+
+      scanner.setRange(split.getRange());
+      scannerBase = scanner;
+
+    }
+
+    Collection<IteratorSetting.Column> columns = split.getFetchedColumns();
+    if (columns == null) {
+      columns = tableConfig.getFetchedColumns();
+    }
+
+    // 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());
+        scannerBase.fetchColumn(c.getFirst(), c.getSecond());
+      } else {
+        log.debug("Fetching column family " + c.getFirst());
+        scannerBase.fetchColumnFamily(c.getFirst());
+      }
+    }
+
+    SamplerConfiguration samplerConfig = split.getSamplerConfiguration();
+    if (samplerConfig == null) {
+      samplerConfig = tableConfig.getSamplerConfiguration();
+    }
+
+    if (samplerConfig != null) {
+      scannerBase.setSamplerConfiguration(samplerConfig);
+    }
+
+    Map<String,String> executionHints = split.getExecutionHints();
+    if (executionHints == null || executionHints.isEmpty()) {
+      executionHints = tableConfig.getExecutionHints();
+    }
+
+    if (executionHints != null) {
+      scannerBase.setExecutionHints(executionHints);
+    }
+
+    scannerIterator = scannerBase.iterator();
+    numKeysRead = 0;
+  }
+
+  @Override
+  public void close() {
+    if (scannerBase != null) {
+      scannerBase.close();
+    }
+    if (client != null) {
+      client.close();
+    }
+  }
+
+  @Override
+  public float getProgress() {
+    if (numKeysRead > 0 && currentKey == null)
+      return 1.0f;
+    return split.getProgress(currentKey);
+  }
+
+  @Override
+  public K getCurrentKey() {
+    return currentK;
+  }
+
+  @Override
+  public V getCurrentValue() {
+    return currentV;
+  }
+
+  /**
+   * Check whether a configuration is fully configured to be used with an Accumulo
+   * {@link org.apache.hadoop.mapreduce.InputFormat}.
+   */
+  private static void validateOptions(JobContext context, Class<?> callingClass)
+      throws IOException {
+    try (AccumuloClient client = InputConfigurator.createClient(callingClass,
+        context.getConfiguration())) {
+      InputConfigurator.validatePermissions(callingClass, context.getConfiguration(), client);
+    }
+  }
+
+  private static Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext context,
+      Table.ID tableId, List<Range> ranges, Class<?> callingClass)
+      throws TableNotFoundException, AccumuloException {
+    try (AccumuloClient client = createClient(context, callingClass)) {
+      return InputConfigurator.binOffline(tableId, ranges, (ClientContext) client);
+    }
+  }
+
+  public static List<InputSplit> getSplits(JobContext context, Class<?> callingClass)
+      throws IOException {
+    validateOptions(context, callingClass);
+    Random random = new SecureRandom();
+    LinkedList<InputSplit> splits = new LinkedList<>();
+    try (AccumuloClient client = createClient(context, callingClass)) {
+      Map<String,InputTableConfig> tableConfigs = InputConfigurator
+          .getInputTableConfigs(callingClass, context.getConfiguration());
+      for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
+
+        String tableName = tableConfigEntry.getKey();
+        InputTableConfig tableConfig = tableConfigEntry.getValue();
+
+        ClientContext clientContext = (ClientContext) client;
+        Table.ID tableId;
+        // resolve table name to id once, and use id from this point forward
+        try {
+          tableId = Tables.getTableId(clientContext, tableName);
+        } catch (TableNotFoundException e) {
+          throw new IOException(e);
+        }
+
+        boolean batchScan = InputConfigurator.isBatchScan(callingClass, context.getConfiguration());
+        boolean supportBatchScan = !(tableConfig.isOfflineScan()
+            || tableConfig.shouldUseIsolatedScanners() || tableConfig.shouldUseLocalIterators());
+        if (batchScan && !supportBatchScan)
+          throw new IllegalArgumentException("BatchScanner optimization not available for offline"
+              + " scan, isolated, or local iterators");
+
+        boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
+        if (batchScan && !autoAdjust)
+          throw new IllegalArgumentException(
+              "AutoAdjustRanges must be enabled when using BatchScanner optimization");
+
+        List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges())
+            : tableConfig.getRanges();
+        if (ranges.isEmpty()) {
+          ranges = new ArrayList<>(1);
+          ranges.add(new Range());
+        }
+
+        // get the metadata information for these ranges
+        Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
+        TabletLocator tl;
+        try {
+          if (tableConfig.isOfflineScan()) {
+            binnedRanges = binOfflineTable(context, tableId, ranges, callingClass);
+            while (binnedRanges == null) {
+              // Some tablets were still online, try again
+              // sleep randomly between 100 and 200 ms
+              sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
+              binnedRanges = binOfflineTable(context, tableId, ranges, callingClass);
+
+            }
+          } else {
+            tl = InputConfigurator.getTabletLocator(callingClass, context.getConfiguration(),
+                tableId);
+            // its possible that the cache could contain complete, but old information about a
+            // tables tablets... so clear it
+            tl.invalidateCache();
+
+            while (!tl.binRanges(clientContext, ranges, binnedRanges).isEmpty()) {
+              String tableIdStr = tableId.canonicalID();
+              if (!Tables.exists(clientContext, tableId))
+                throw new TableDeletedException(tableIdStr);
+              if (Tables.getTableState(clientContext, tableId) == TableState.OFFLINE)
+                throw new TableOfflineException(Tables.getTableOfflineMsg(clientContext, tableId));
+              binnedRanges.clear();
+              log.warn("Unable to locate bins for specified ranges. Retrying.");
+              // sleep randomly between 100 and 200 ms
+              sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
+              tl.invalidateCache();
+            }
+          }
+        } catch (Exception e) {
+          throw new IOException(e);
+        }
+
+        // all of this code will add either range per each locations or split ranges and add
+        // range-location split
+        // Map from Range to Array of Locations, we only use this if we're don't split
+        HashMap<Range,ArrayList<String>> splitsToAdd = null;
+
+        if (!autoAdjust)
+          splitsToAdd = new HashMap<>();
+
+        HashMap<String,String> hostNameCache = new HashMap<>();
+        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();
+            if (batchScan) {
+              // group ranges by tablet to be read by a BatchScanner
+              ArrayList<Range> clippedRanges = new ArrayList<>();
+              for (Range r : extentRanges.getValue())
+                clippedRanges.add(ke.clip(r));
+              BatchInputSplit split = new BatchInputSplit(tableName, tableId, clippedRanges,
+                  new String[] {location});
+              SplitUtils.updateSplit(split, tableConfig);
+
+              splits.add(split);
+            } else {
+              // not grouping by tablet
+              for (Range r : extentRanges.getValue()) {
+                if (autoAdjust) {
+                  // divide ranges into smaller ranges, based on the tablets
+                  RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+                      ke.clip(r), new String[] {location});
+                  SplitUtils.updateSplit(split, tableConfig);
+                  split.setOffline(tableConfig.isOfflineScan());
+                  split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
+                  split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
+                  splits.add(split);
+                } else {
+                  // don't divide ranges
+                  ArrayList<String> locations = splitsToAdd.get(r);
+                  if (locations == null)
+                    locations = new ArrayList<>(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.canonicalID(),
+                entry.getKey(), entry.getValue().toArray(new String[0]));
+            SplitUtils.updateSplit(split, tableConfig);
+            split.setOffline(tableConfig.isOfflineScan());
+            split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
+            split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
+
+            splits.add(split);
+          }
+      }
+    }
+    return splits;
+  }
+
+  /**
+   * Gets the {@link ClientInfo} from the configuration
+   */
+  private static ClientInfo getClientInfo(JobContext context, Class<?> callingClass) {
+    return InputConfigurator.getClientInfo(callingClass, context.getConfiguration());
+  }
+
+  /**
+   * Creates {@link AccumuloClient} from the configuration
+   */
+  private static AccumuloClient createClient(JobContext context, Class<?> callingClass) {
+    return Accumulo.newClient().from(getClientInfo(context, callingClass).getProperties()).build();
+  }
+}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordWriter.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordWriter.java
new file mode 100644
index 0000000..3fafdb6
--- /dev/null
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordWriter.java
@@ -0,0 +1,213 @@
+/*
+ * 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.hadoopImpl.mapreduce;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+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.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.security.SecurityErrorCode;
+import org.apache.accumulo.core.data.ColumnUpdate;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.hadoopImpl.mapreduce.lib.OutputConfigurator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
+ */
+public class AccumuloRecordWriter extends RecordWriter<Text,Mutation> {
+  // class to serialize configuration under in the job
+  private static final Class<AccumuloOutputFormat> CLASS = AccumuloOutputFormat.class;
+  private static final Logger log = LoggerFactory.getLogger(AccumuloRecordWriter.class);
+  private MultiTableBatchWriter mtbw = null;
+  private final HashMap<Text,BatchWriter> bws;
+  private Text defaultTableName;
+
+  private boolean simulate;
+  private boolean createTables;
+
+  private long mutCount = 0;
+  private long valCount = 0;
+
+  private AccumuloClient client;
+
+  public AccumuloRecordWriter(TaskAttemptContext context) {
+    Configuration conf = context.getConfiguration();
+    this.simulate = OutputConfigurator.getSimulationMode(CLASS, conf);
+    this.createTables = OutputConfigurator.canCreateTables(CLASS, conf);
+
+    if (simulate)
+      log.info("Simulating output only. No writes to tables will occur");
+
+    this.bws = new HashMap<>();
+
+    String tname = OutputConfigurator.getDefaultTableName(CLASS, conf);
+    this.defaultTableName = (tname == null) ? null : new Text(tname);
+
+    if (!simulate) {
+      this.client = Accumulo.newClient()
+          .from(OutputConfigurator.getClientInfo(CLASS, conf).getProperties()).build();
+      mtbw = client.createMultiTableBatchWriter();
+    }
+  }
+
+  /**
+   * Push a mutation into a table. If table is null, the defaultTable will be used. If
+   * OutputFormatBuilder#createTables() 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) {
+        log.error("Could not add table '" + table + "'", e);
+        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 && !client.tableOperations().exists(table)) {
+      try {
+        client.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 | 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 {
+    log.debug("mutations written: " + mutCount + ", values written: " + valCount);
+    if (simulate)
+      return;
+
+    try {
+      mtbw.close();
+    } catch (MutationsRejectedException e) {
+      if (e.getSecurityErrorCodes().size() >= 0) {
+        HashMap<String,Set<SecurityErrorCode>> tables = new HashMap<>();
+        for (Map.Entry<TabletId,Set<SecurityErrorCode>> ke : e.getSecurityErrorCodes().entrySet()) {
+          String tableId = ke.getKey().getTableId().toString();
+          Set<SecurityErrorCode> secCodes = tables.get(tableId);
+          if (secCodes == null) {
+            secCodes = new HashSet<>();
+            tables.put(tableId, 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());
+      }
+      throw new IOException(e);
+    } finally {
+      client.close();
+    }
+  }
+}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/FileOutputFormatBuilderImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/FileOutputFormatBuilderImpl.java
index 5e06e1f..9813f01 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/FileOutputFormatBuilderImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/FileOutputFormatBuilderImpl.java
@@ -16,15 +16,6 @@
  */
 package org.apache.accumulo.hadoopImpl.mapreduce;
 
-import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloFileOutputFormatImpl.setCompressionType;
-import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloFileOutputFormatImpl.setDataBlockSize;
-import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloFileOutputFormatImpl.setFileBlockSize;
-import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloFileOutputFormatImpl.setIndexBlockSize;
-import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloFileOutputFormatImpl.setReplication;
-import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloFileOutputFormatImpl.setSampler;
-import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloFileOutputFormatImpl.setSummarizers;
-import static org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.setOutputPath;
-
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -34,6 +25,8 @@ import java.util.Optional;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.hadoop.mapreduce.FileOutputFormatBuilder;
+import org.apache.accumulo.hadoopImpl.mapreduce.lib.FileOutputConfigurator;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
@@ -41,6 +34,7 @@ import org.apache.hadoop.mapreduce.Job;
 public class FileOutputFormatBuilderImpl<T> implements FileOutputFormatBuilder,
     FileOutputFormatBuilder.PathParams<T>, FileOutputFormatBuilder.OutputOptions<T> {
 
+  Class<?> callingClass;
   Path outputPath;
   Optional<String> comp = Optional.empty();
   Optional<Long> dataBlockSize = Optional.empty();
@@ -50,6 +44,10 @@ public class FileOutputFormatBuilderImpl<T> implements FileOutputFormatBuilder,
   Optional<SamplerConfiguration> sampler = Optional.empty();
   Collection<SummarizerConfiguration> summarizers = Collections.emptySet();
 
+  public FileOutputFormatBuilderImpl(Class<?> callingClass) {
+    this.callingClass = callingClass;
+  }
+
   @Override
   public OutputOptions<T> outputPath(Path path) {
     this.outputPath = Objects.requireNonNull(path);
@@ -110,46 +108,31 @@ public class FileOutputFormatBuilderImpl<T> implements FileOutputFormatBuilder,
   }
 
   private void store(Job job) {
-    setOutputPath(job, outputPath);
+    org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.setOutputPath(job, outputPath);
+    _store(job.getConfiguration());
+  }
+
+  private void _store(Configuration conf) {
     if (comp.isPresent())
-      setCompressionType(job, comp.get());
+      FileOutputConfigurator.setCompressionType(callingClass, conf, comp.get());
     if (dataBlockSize.isPresent())
-      setDataBlockSize(job, dataBlockSize.get());
+      FileOutputConfigurator.setDataBlockSize(callingClass, conf, dataBlockSize.get());
     if (fileBlockSize.isPresent())
-      setFileBlockSize(job, fileBlockSize.get());
+      FileOutputConfigurator.setFileBlockSize(callingClass, conf, fileBlockSize.get());
     if (indexBlockSize.isPresent())
-      setIndexBlockSize(job, indexBlockSize.get());
+      FileOutputConfigurator.setIndexBlockSize(callingClass, conf, indexBlockSize.get());
     if (replication.isPresent())
-      setReplication(job, replication.get());
+      FileOutputConfigurator.setReplication(callingClass, conf, replication.get());
     if (sampler.isPresent())
-      setSampler(job, sampler.get());
+      FileOutputConfigurator.setSampler(callingClass, conf, sampler.get());
     if (summarizers.size() > 0)
-      setSummarizers(job, summarizers.toArray(new SummarizerConfiguration[0]));
+      FileOutputConfigurator.setSummarizers(callingClass, conf,
+          summarizers.toArray(new SummarizerConfiguration[0]));
   }
 
   private void store(JobConf job) {
     org.apache.hadoop.mapred.FileOutputFormat.setOutputPath(job, outputPath);
-    if (comp.isPresent())
-      org.apache.accumulo.hadoopImpl.mapred.AccumuloFileOutputFormatImpl.setCompressionType(job,
-          comp.get());
-    if (dataBlockSize.isPresent())
-      org.apache.accumulo.hadoopImpl.mapred.AccumuloFileOutputFormatImpl.setDataBlockSize(job,
-          dataBlockSize.get());
-    if (fileBlockSize.isPresent())
-      org.apache.accumulo.hadoopImpl.mapred.AccumuloFileOutputFormatImpl.setFileBlockSize(job,
-          fileBlockSize.get());
-    if (indexBlockSize.isPresent())
-      org.apache.accumulo.hadoopImpl.mapred.AccumuloFileOutputFormatImpl.setIndexBlockSize(job,
-          indexBlockSize.get());
-    if (replication.isPresent())
-      org.apache.accumulo.hadoopImpl.mapred.AccumuloFileOutputFormatImpl.setReplication(job,
-          replication.get());
-    if (sampler.isPresent())
-      org.apache.accumulo.hadoopImpl.mapred.AccumuloFileOutputFormatImpl.setSampler(job,
-          sampler.get());
-    if (summarizers.size() > 0)
-      org.apache.accumulo.hadoopImpl.mapred.AccumuloFileOutputFormatImpl.setSummarizers(job,
-          summarizers.toArray(new SummarizerConfiguration[0]));
+    _store(job);
   }
 
 }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBase.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBase.java
deleted file mode 100644
index f067132..0000000
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBase.java
+++ /dev/null
@@ -1,325 +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.hadoopImpl.mapreduce;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-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.ScannerBase;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.hadoopImpl.mapreduce.lib.InputConfigurator;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-public abstract class InputFormatBase extends AbstractInputFormat {
-
-  /**
-   * Gets the table name from the configuration.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return the table name
-   * @since 1.5.0
-   * @see #setInputTableName(Job, String)
-   */
-  protected static String getInputTableName(JobContext context) {
-    return InputConfigurator.getInputTableName(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * 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(Job job, String tableName) {
-    InputConfigurator.setInputTableName(CLASS, job.getConfiguration(), tableName);
-  }
-
-  /**
-   * Sets the input ranges to scan for the single input table associated with this job.
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param ranges
-   *          the ranges that will be mapped over
-   * @see TableOperations#splitRangeByTablets(String, Range, int)
-   * @since 1.5.0
-   */
-  public static void setRanges(Job job, Collection<Range> ranges) {
-    InputConfigurator.setRanges(CLASS, job.getConfiguration(), ranges);
-  }
-
-  /**
-   * Gets the ranges to scan over from a job.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return the ranges
-   * @since 1.5.0
-   * @see #setRanges(Job, Collection)
-   */
-  protected static List<Range> getRanges(JobContext context) throws IOException {
-    return InputConfigurator.getRanges(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this
-   * configuration.
-   */
-  protected static List<IteratorSetting> getIterators(JobContext context) {
-    return InputConfigurator.getIterators(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * 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(Job, Collection)
-   * @since 1.5.0
-   */
-  public static void setAutoAdjustRanges(Job job, boolean enableFeature) {
-    InputConfigurator.setAutoAdjustRanges(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has auto-adjust ranges enabled. Must be enabled when
-   * {@link #setBatchScan(Job, boolean)} is true.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return false if the feature is disabled, true otherwise
-   * @since 1.5.0
-   * @see #setAutoAdjustRanges(Job, boolean)
-   */
-  protected static boolean getAutoAdjustRanges(JobContext context) {
-    return InputConfigurator.getAutoAdjustRanges(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * 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(Job job, boolean enableFeature) {
-    InputConfigurator.setScanIsolation(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has isolation enabled.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setScanIsolation(Job, boolean)
-   */
-  protected static boolean isIsolated(JobContext context) {
-    return InputConfigurator.isIsolated(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * 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(Job job, boolean enableFeature) {
-    InputConfigurator.setLocalIterators(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration uses local iterators.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setLocalIterators(Job, boolean)
-   */
-  protected static boolean usesLocalIterators(JobContext context) {
-    return InputConfigurator.usesLocalIterators(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * 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(Job job, boolean enableFeature) {
-    InputConfigurator.setOfflineTableScan(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the offline table scan feature enabled.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setOfflineTableScan(Job, boolean)
-   */
-  protected static boolean isOfflineScan(JobContext context) {
-    return InputConfigurator.isOfflineScan(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Controls the use of the {@link org.apache.accumulo.core.client.BatchScanner} in this job. Using
-   * this feature will group Ranges by their source tablet, producing an InputSplit per tablet
-   * rather than per Range. This batching helps to reduce overhead when querying a large number of
-   * small ranges. (ex: when doing quad-tree decomposition for spatial queries)
-   * <p>
-   * In order to achieve good locality of InputSplits this option always clips the input Ranges to
-   * tablet boundaries. This may result in one input Range contributing to several InputSplits.
-   * <p>
-   * Note: that the value of {@link #setAutoAdjustRanges(Job, boolean)} is ignored and is assumed to
-   * be true when BatchScan option is enabled.
-   * <p>
-   * This configuration is incompatible with:
-   * <ul>
-   * <li>{@link #setOfflineTableScan(org.apache.hadoop.mapreduce.Job, boolean)}</li>
-   * <li>{@link #setLocalIterators(org.apache.hadoop.mapreduce.Job, boolean)}</li>
-   * <li>{@link #setScanIsolation(org.apache.hadoop.mapreduce.Job, boolean)}</li>
-   * </ul>
-   * <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.7.0
-   */
-  public static void setBatchScan(Job job, boolean enableFeature) {
-    InputConfigurator.setBatchScan(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the {@link org.apache.accumulo.core.client.BatchScanner}
-   * feature enabled.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @since 1.7.0
-   * @see #setBatchScan(Job, boolean)
-   */
-  protected static boolean isBatchScan(JobContext context) {
-    return InputConfigurator.isBatchScan(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Causes input format to read sample data. If sample data was created using a different
-   * configuration or a tables sampler configuration changes while reading data, then the input
-   * format will throw an error.
-   *
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param samplerConfig
-   *          The sampler configuration that sample must have been created with inorder for reading
-   *          sample data to succeed.
-   *
-   * @since 1.8.0
-   * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
-   */
-  public static void setSamplerConfiguration(Job job, SamplerConfiguration samplerConfig) {
-    InputConfigurator.setSamplerConfiguration(CLASS, job.getConfiguration(), samplerConfig);
-  }
-
-  /**
-   * Set these execution hints on scanners created for input splits. See
-   * {@link ScannerBase#setExecutionHints(java.util.Map)}
-   *
-   * @since 2.0.0
-   */
-  public static void setExecutionHints(Job job, Map<String,String> hints) {
-    InputConfigurator.setExecutionHints(CLASS, job.getConfiguration(), hints);
-  }
-
-  public abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
-
-    @Override
-    protected List<IteratorSetting> contextIterators(TaskAttemptContext context) {
-      return getIterators(context);
-    }
-  }
-}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
index 0af63d8..119a921 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
@@ -35,6 +35,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.hadoop.mapreduce.InputFormatBuilder;
 import org.apache.accumulo.hadoopImpl.mapreduce.lib.InputConfigurator;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
 
@@ -177,40 +178,43 @@ public class InputFormatBuilderImpl<T>
    * Final builder method for mapreduce configuration
    */
   private void store(Job job) throws AccumuloException, AccumuloSecurityException {
-    AbstractInputFormat.setClientInfo(job, clientInfo);
+    _store(job.getConfiguration());
+  }
+
+  private void _store(Configuration conf) throws AccumuloException, AccumuloSecurityException {
+    InputConfigurator.setClientInfo(callingClass, conf, clientInfo);
     if (tableConfigMap.size() == 0) {
       throw new IllegalArgumentException("At least one Table must be configured for job.");
     }
     // if only one table use the single table configuration method
     if (tableConfigMap.size() == 1) {
       Map.Entry<String,InputTableConfig> entry = tableConfigMap.entrySet().iterator().next();
-      InputFormatBase.setInputTableName(job, entry.getKey());
+      InputConfigurator.setInputTableName(callingClass, conf, entry.getKey());
       InputTableConfig config = entry.getValue();
       if (!config.getScanAuths().isPresent())
         config.setScanAuths(getUserAuths(clientInfo));
-      AbstractInputFormat.setScanAuthorizations(job, config.getScanAuths().get());
+      InputConfigurator.setScanAuthorizations(callingClass, conf, config.getScanAuths().get());
       // all optional values
       if (config.getContext().isPresent())
-        AbstractInputFormat.setClassLoaderContext(job, config.getContext().get());
+        InputConfigurator.setClassLoaderContext(callingClass, conf, config.getContext().get());
       if (config.getRanges().size() > 0)
-        InputFormatBase.setRanges(job, config.getRanges());
+        InputConfigurator.setRanges(callingClass, conf, config.getRanges());
       if (config.getIterators().size() > 0)
-        InputConfigurator.writeIteratorsToConf(callingClass, job.getConfiguration(),
-            config.getIterators());
+        InputConfigurator.writeIteratorsToConf(callingClass, conf, config.getIterators());
       if (config.getFetchedColumns().size() > 0)
-        InputConfigurator.fetchColumns(callingClass, job.getConfiguration(),
-            config.getFetchedColumns());
+        InputConfigurator.fetchColumns(callingClass, conf, config.getFetchedColumns());
       if (config.getSamplerConfiguration() != null)
-        InputFormatBase.setSamplerConfiguration(job, config.getSamplerConfiguration());
+        InputConfigurator.setSamplerConfiguration(callingClass, conf,
+            config.getSamplerConfiguration());
       if (config.getExecutionHints().size() > 0)
-        InputFormatBase.setExecutionHints(job, config.getExecutionHints());
-      InputFormatBase.setAutoAdjustRanges(job, config.shouldAutoAdjustRanges());
-      InputFormatBase.setScanIsolation(job, config.shouldUseIsolatedScanners());
-      InputFormatBase.setLocalIterators(job, config.shouldUseLocalIterators());
-      InputFormatBase.setOfflineTableScan(job, config.isOfflineScan());
-      InputFormatBase.setBatchScan(job, config.shouldBatchScan());
+        InputConfigurator.setExecutionHints(callingClass, conf, config.getExecutionHints());
+      InputConfigurator.setAutoAdjustRanges(callingClass, conf, config.shouldAutoAdjustRanges());
+      InputConfigurator.setScanIsolation(callingClass, conf, config.shouldUseIsolatedScanners());
+      InputConfigurator.setLocalIterators(callingClass, conf, config.shouldUseLocalIterators());
+      InputConfigurator.setOfflineTableScan(callingClass, conf, config.isOfflineScan());
+      InputConfigurator.setBatchScan(callingClass, conf, config.shouldBatchScan());
     } else {
-      InputConfigurator.setInputTableConfigs(callingClass, job.getConfiguration(), tableConfigMap);
+      InputConfigurator.setInputTableConfigs(callingClass, conf, tableConfigMap);
     }
   }
 
@@ -218,50 +222,7 @@ public class InputFormatBuilderImpl<T>
    * Final builder method for legacy mapred configuration
    */
   private void store(JobConf jobConf) throws AccumuloException, AccumuloSecurityException {
-    org.apache.accumulo.hadoopImpl.mapred.AbstractInputFormat.setClientInfo(jobConf, clientInfo);
-    if (tableConfigMap.size() == 0) {
-      throw new IllegalArgumentException("At least one Table must be configured for job.");
-    }
-    // if only one table use the single table configuration method
-    if (tableConfigMap.size() == 1) {
-      Map.Entry<String,InputTableConfig> entry = tableConfigMap.entrySet().iterator().next();
-      org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setInputTableName(jobConf,
-          entry.getKey());
-      InputTableConfig config = entry.getValue();
-      if (!config.getScanAuths().isPresent())
-        config.setScanAuths(getUserAuths(clientInfo));
-      org.apache.accumulo.hadoopImpl.mapred.AbstractInputFormat.setScanAuthorizations(jobConf,
-          config.getScanAuths().get());
-      // all optional values
-      if (config.getContext().isPresent())
-        org.apache.accumulo.hadoopImpl.mapred.AbstractInputFormat.setClassLoaderContext(jobConf,
-            config.getContext().get());
-      if (config.getRanges().size() > 0)
-        org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setRanges(jobConf,
-            config.getRanges());
-      if (config.getIterators().size() > 0)
-        InputConfigurator.writeIteratorsToConf(callingClass, jobConf, config.getIterators());
-      if (config.getFetchedColumns().size() > 0)
-        InputConfigurator.fetchColumns(callingClass, jobConf, config.getFetchedColumns());
-      if (config.getSamplerConfiguration() != null)
-        org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setSamplerConfiguration(jobConf,
-            config.getSamplerConfiguration());
-      if (config.getExecutionHints().size() > 0)
-        org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setExecutionHints(jobConf,
-            config.getExecutionHints());
-      org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setAutoAdjustRanges(jobConf,
-          config.shouldAutoAdjustRanges());
-      org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setScanIsolation(jobConf,
-          config.shouldUseIsolatedScanners());
-      org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setLocalIterators(jobConf,
-          config.shouldUseLocalIterators());
-      org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setOfflineTableScan(jobConf,
-          config.isOfflineScan());
-      org.apache.accumulo.hadoopImpl.mapred.InputFormatBase.setBatchScan(jobConf,
-          config.shouldBatchScan());
-    } else {
-      InputConfigurator.setInputTableConfigs(callingClass, jobConf, tableConfigMap);
-    }
+    _store(jobConf);
   }
 
   private Authorizations getUserAuths(ClientInfo clientInfo)
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputTableConfig.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputTableConfig.java
index 9d27d6f..073d366 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputTableConfig.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputTableConfig.java
@@ -271,10 +271,6 @@ public class InputTableConfig implements Writable {
 
   /**
    * Set the sampler configuration to use when reading from the data.
-   *
-   * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
-   * @see InputFormatBase#setSamplerConfiguration(org.apache.hadoop.mapreduce.Job,
-   *      SamplerConfiguration)
    */
   public void setSamplerConfiguration(SamplerConfiguration samplerConfiguration) {
     this.samplerConfig = samplerConfiguration;
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/OutputFormatBuilderImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/OutputFormatBuilderImpl.java
index 217f01f..b0c7656 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/OutputFormatBuilderImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/OutputFormatBuilderImpl.java
@@ -16,22 +16,20 @@
  */
 package org.apache.accumulo.hadoopImpl.mapreduce;
 
-import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloOutputFormatImpl.setClientInfo;
-import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloOutputFormatImpl.setCreateTables;
-import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloOutputFormatImpl.setDefaultTableName;
-import static org.apache.accumulo.hadoopImpl.mapreduce.AccumuloOutputFormatImpl.setSimulationMode;
-
 import java.util.Objects;
 import java.util.Optional;
 import java.util.Properties;
 
 import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.hadoop.mapreduce.OutputFormatBuilder;
+import org.apache.accumulo.hadoopImpl.mapreduce.lib.OutputConfigurator;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
 
 public class OutputFormatBuilderImpl<T>
     implements OutputFormatBuilder.ClientParams<T>, OutputFormatBuilder.OutputOptions<T> {
+  private final Class<?> callingClass;
   ClientInfo clientInfo;
 
   // optional values
@@ -39,6 +37,10 @@ public class OutputFormatBuilderImpl<T>
   boolean createTables = false;
   boolean simulationMode = false;
 
+  public OutputFormatBuilderImpl(Class<?> callingClass) {
+    this.callingClass = callingClass;
+  }
+
   @Override
   public OutputFormatBuilder.OutputOptions<T> clientProperties(Properties clientProperties) {
     this.clientInfo = ClientInfo
@@ -76,24 +78,19 @@ public class OutputFormatBuilderImpl<T>
   }
 
   private void store(Job job) {
-    setClientInfo(job, clientInfo);
-    if (defaultTableName.isPresent())
-      setDefaultTableName(job, defaultTableName.get());
-    setCreateTables(job, createTables);
-    setSimulationMode(job, simulationMode);
+    _store(job.getConfiguration());
   }
 
-  private void store(JobConf jobConf) {
-    org.apache.accumulo.hadoopImpl.mapred.AccumuloOutputFormatImpl.setClientInfo(jobConf,
-        clientInfo);
+  private void _store(Configuration conf) {
+    OutputConfigurator.setClientInfo(callingClass, conf, clientInfo);
     if (defaultTableName.isPresent())
-      org.apache.accumulo.hadoopImpl.mapred.AccumuloOutputFormatImpl.setDefaultTableName(jobConf,
-          defaultTableName.get());
-    org.apache.accumulo.hadoopImpl.mapred.AccumuloOutputFormatImpl.setCreateTables(jobConf,
-        createTables);
-    org.apache.accumulo.hadoopImpl.mapred.AccumuloOutputFormatImpl.setSimulationMode(jobConf,
-        simulationMode);
+      OutputConfigurator.setDefaultTableName(callingClass, conf, defaultTableName.get());
+    OutputConfigurator.setCreateTables(callingClass, conf, createTables);
+    OutputConfigurator.setSimulationMode(callingClass, conf, simulationMode);
+  }
 
+  private void store(JobConf jobConf) {
+    _store(jobConf);
   }
 
 }


Mime
View raw message