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

[04/12] ACCUMULO-1880 create mapreduce module

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4dfcb9de/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java
new file mode 100644
index 0000000..ce84209
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapreduce.lib.impl;
+
+import java.util.Arrays;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * @since 1.6.0
+ */
+public class FileOutputConfigurator extends ConfiguratorBase {
+
+  /**
+   * Configuration keys for {@link AccumuloConfiguration}.
+   * 
+   * @since 1.6.0
+   */
+  public static enum Opts {
+    ACCUMULO_PROPERTIES;
+  }
+
+  /**
+   * The supported Accumulo properties we set in this OutputFormat, that change the behavior of the RecordWriter.<br />
+   * These properties correspond to the supported public static setter methods available to this class.
+   * 
+   * @param property
+   *          the Accumulo property to check
+   * @since 1.6.0
+   */
+  protected static Boolean isSupportedAccumuloProperty(Property property) {
+    switch (property) {
+      case TABLE_FILE_COMPRESSION_TYPE:
+      case TABLE_FILE_COMPRESSED_BLOCK_SIZE:
+      case TABLE_FILE_BLOCK_SIZE:
+      case TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX:
+      case TABLE_FILE_REPLICATION:
+        return true;
+      default:
+        return false;
+    }
+  }
+
+  /**
+   * Helper for transforming Accumulo configuration properties into something that can be stored safely inside the Hadoop Job configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param property
+   *          the supported Accumulo property
+   * @param value
+   *          the value of the property to set
+   * @since 1.6.0
+   */
+  private static <T> void setAccumuloProperty(Class<?> implementingClass, Configuration conf, Property property, T value) {
+    if (isSupportedAccumuloProperty(property)) {
+      String val = String.valueOf(value);
+      if (property.getType().isValidFormat(val))
+        conf.set(enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + "." + property.getKey(), val);
+      else
+        throw new IllegalArgumentException("Value is not appropriate for property type '" + property.getType() + "'");
+    } else
+      throw new IllegalArgumentException("Unsupported configuration property " + property.getKey());
+  }
+
+  /**
+   * This helper method provides an AccumuloConfiguration object constructed from the Accumulo defaults, and overridden with Accumulo properties that have been
+   * stored in the Job's configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @since 1.6.0
+   */
+  public static AccumuloConfiguration getAccumuloConfiguration(Class<?> implementingClass, Configuration conf) {
+    String prefix = enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + ".";
+    ConfigurationCopy acuConf = new ConfigurationCopy(AccumuloConfiguration.getDefaultConfiguration());
+    for (Entry<String,String> entry : conf)
+      if (entry.getKey().startsWith(prefix))
+        acuConf.set(Property.getPropertyByKey(entry.getKey().substring(prefix.length())), entry.getValue());
+    return acuConf;
+  }
+
+  /**
+   * Sets the compression type to use for data blocks. Specifying a compression may require additional libraries to be available to your Job.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param compressionType
+   *          one of "none", "gz", "lzo", or "snappy"
+   * @since 1.6.0
+   */
+  public static void setCompressionType(Class<?> implementingClass, Configuration conf, String compressionType) {
+    if (compressionType == null || !Arrays.asList("none", "gz", "lzo", "snappy").contains(compressionType))
+      throw new IllegalArgumentException("Compression type must be one of: none, gz, lzo, snappy");
+    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSION_TYPE, 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 implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param dataBlockSize
+   *          the block size, in bytes
+   * @since 1.6.0
+   */
+  public static void setDataBlockSize(Class<?> implementingClass, Configuration conf, long dataBlockSize) {
+    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE, dataBlockSize);
+  }
+
+  /**
+   * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by the underlying file system.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param fileBlockSize
+   *          the block size, in bytes
+   * @since 1.6.0
+   */
+  public static void setFileBlockSize(Class<?> implementingClass, Configuration conf, long fileBlockSize) {
+    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_BLOCK_SIZE, 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 implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param indexBlockSize
+   *          the block size, in bytes
+   * @since 1.6.0
+   */
+  public static void setIndexBlockSize(Class<?> implementingClass, Configuration conf, long indexBlockSize) {
+    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX, indexBlockSize);
+  }
+
+  /**
+   * Sets the file system replication factor for the resulting file, overriding the file system default.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param replication
+   *          the number of replicas for produced files
+   * @since 1.6.0
+   */
+  public static void setReplication(Class<?> implementingClass, Configuration conf, int replication) {
+    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_REPLICATION, replication);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4dfcb9de/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
new file mode 100644
index 0000000..7657c3c
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
@@ -0,0 +1,796 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapreduce.lib.impl;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringTokenizer;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientSideIteratorScanner;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IsolatedScanner;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.RowIterator;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.TabletLocator;
+import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
+import org.apache.accumulo.core.client.mock.MockTabletLocator;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.collect.Maps;
+
+/**
+ * @since 1.6.0
+ */
+public class InputConfigurator extends ConfiguratorBase {
+
+  /**
+   * Configuration keys for {@link Scanner}.
+   * 
+   * @since 1.6.0
+   */
+  public static enum ScanOpts {
+    TABLE_NAME, AUTHORIZATIONS, RANGES, COLUMNS, ITERATORS, TABLE_CONFIGS
+  }
+
+  /**
+   * Configuration keys for various features.
+   * 
+   * @since 1.6.0
+   */
+  public static enum Features {
+    AUTO_ADJUST_RANGES, SCAN_ISOLATION, USE_LOCAL_ITERATORS, SCAN_OFFLINE
+  }
+
+  /**
+   * Sets the name of the input table, over which this job will scan.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param tableName
+   *          the table to use when the tablename is null in the write call
+   * @since 1.6.0
+   */
+  public static void setInputTableName(Class<?> implementingClass, Configuration conf, String tableName) {
+    checkArgument(tableName != null, "tableName is null");
+    conf.set(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME), tableName);
+  }
+
+  /**
+   * Sets the name of the input table, over which this job will scan.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @since 1.6.0
+   */
+  public static String getInputTableName(Class<?> implementingClass, Configuration conf) {
+    return conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME));
+  }
+
+  /**
+   * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param auths
+   *          the user's authorizations
+   * @since 1.6.0
+   */
+  public static void setScanAuthorizations(Class<?> implementingClass, Configuration conf, Authorizations auths) {
+    if (auths != null && !auths.isEmpty())
+      conf.set(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS), auths.serialize());
+  }
+
+  /**
+   * Gets the authorizations to set for the scans from the configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return the Accumulo scan authorizations
+   * @since 1.6.0
+   * @see #setScanAuthorizations(Class, Configuration, Authorizations)
+   */
+  public static Authorizations getScanAuthorizations(Class<?> implementingClass, Configuration conf) {
+    String authString = conf.get(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS));
+    return authString == null ? Authorizations.EMPTY : new Authorizations(authString.getBytes(StandardCharsets.UTF_8));
+  }
+
+  /**
+   * Sets the input ranges to scan on all input tables for this job. If not set, the entire table will be scanned.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param ranges
+   *          the ranges that will be mapped over
+   * @throws IllegalArgumentException
+   *           if the ranges cannot be encoded into base 64
+   * @since 1.6.0
+   */
+  public static void setRanges(Class<?> implementingClass, Configuration conf, Collection<Range> ranges) {
+    checkArgument(ranges != null, "ranges is null");
+
+    ArrayList<String> rangeStrings = new ArrayList<String>(ranges.size());
+    try {
+      for (Range r : ranges) {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        r.write(new DataOutputStream(baos));
+        rangeStrings.add(new String(Base64.encodeBase64(baos.toByteArray())));
+      }
+      conf.setStrings(enumToConfKey(implementingClass, ScanOpts.RANGES), rangeStrings.toArray(new String[0]));
+    } catch (IOException ex) {
+      throw new IllegalArgumentException("Unable to encode ranges to Base64", ex);
+    }
+  }
+
+  /**
+   * Gets the ranges to scan over from a job.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return the ranges
+   * @throws IOException
+   *           if the ranges have been encoded improperly
+   * @since 1.6.0
+   * @see #setRanges(Class, Configuration, Collection)
+   */
+  public static List<Range> getRanges(Class<?> implementingClass, Configuration conf) throws IOException {
+
+    Collection<String> encodedRanges = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.RANGES));
+    List<Range> ranges = new ArrayList<Range>();
+    for (String rangeString : encodedRanges) {
+      ByteArrayInputStream bais = new ByteArrayInputStream(Base64.decodeBase64(rangeString.getBytes()));
+      Range range = new Range();
+      range.readFields(new DataInputStream(bais));
+      ranges.add(range);
+    }
+    return ranges;
+  }
+
+  /**
+   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return a list of iterators
+   * @since 1.6.0
+   * @see #addIterator(Class, Configuration, IteratorSetting)
+   */
+  public static List<IteratorSetting> getIterators(Class<?> implementingClass, Configuration conf) {
+    String iterators = conf.get(enumToConfKey(implementingClass, ScanOpts.ITERATORS));
+
+    // If no iterators are present, return an empty list
+    if (iterators == null || iterators.isEmpty())
+      return new ArrayList<IteratorSetting>();
+
+    // Compose the set of iterators encoded in the job configuration
+    StringTokenizer tokens = new StringTokenizer(iterators, StringUtils.COMMA_STR);
+    List<IteratorSetting> list = new ArrayList<IteratorSetting>();
+    try {
+      while (tokens.hasMoreTokens()) {
+        String itstring = tokens.nextToken();
+        ByteArrayInputStream bais = new ByteArrayInputStream(Base64.decodeBase64(itstring.getBytes()));
+        list.add(new IteratorSetting(new DataInputStream(bais)));
+        bais.close();
+      }
+    } catch (IOException e) {
+      throw new IllegalArgumentException("couldn't decode iterator settings");
+    }
+    return list;
+  }
+
+  /**
+   * Restricts the columns that will be mapped over for the single input table on this job.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param columnFamilyColumnQualifierPairs
+   *          a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
+   *          selected. An empty set is the default and is equivalent to scanning the all columns.
+   * @throws IllegalArgumentException
+   *           if the column family is null
+   * @since 1.6.0
+   */
+  public static void fetchColumns(Class<?> implementingClass, Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
+    checkArgument(columnFamilyColumnQualifierPairs != null, "columnFamilyColumnQualifierPairs is null");
+    String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
+    conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings);
+  }
+
+  public static String[] serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
+    checkArgument(columnFamilyColumnQualifierPairs != null, "columnFamilyColumnQualifierPairs is null");
+    ArrayList<String> columnStrings = new ArrayList<String>(columnFamilyColumnQualifierPairs.size());
+    for (Pair<Text,Text> column : columnFamilyColumnQualifierPairs) {
+
+      if (column.getFirst() == null)
+        throw new IllegalArgumentException("Column family can not be null");
+
+      String col = new String(Base64.encodeBase64(TextUtil.getBytes(column.getFirst())), StandardCharsets.UTF_8);
+      if (column.getSecond() != null)
+        col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())), StandardCharsets.UTF_8);
+      columnStrings.add(col);
+    }
+
+    return columnStrings.toArray(new String[0]);
+  }
+
+  /**
+   * Gets the columns to be mapped over from this job.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return a set of columns
+   * @since 1.6.0
+   * @see #fetchColumns(Class, Configuration, Collection)
+   */
+  public static Set<Pair<Text,Text>> getFetchedColumns(Class<?> implementingClass, Configuration conf) {
+    checkArgument(conf != null, "conf is null");
+    String confValue = conf.get(enumToConfKey(implementingClass, ScanOpts.COLUMNS));
+    List<String> serialized = new ArrayList<String>();
+    if (confValue != null) {
+      // Split and include any trailing empty strings to allow empty column families
+      for (String val : confValue.split(",", -1)) {
+        serialized.add(val);
+      }
+    }
+    return deserializeFetchedColumns(serialized);
+  }
+
+  public static Set<Pair<Text,Text>> deserializeFetchedColumns(Collection<String> serialized) {
+    Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
+
+    if (null == serialized) {
+      return columns;
+    }
+
+    for (String col : serialized) {
+      int idx = col.indexOf(":");
+      Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes(StandardCharsets.UTF_8)) : Base64.decodeBase64(col.substring(0, idx).getBytes(
+          StandardCharsets.UTF_8)));
+      Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes(StandardCharsets.UTF_8)));
+      columns.add(new Pair<Text,Text>(cf, cq));
+    }
+    return columns;
+  }
+
+  /**
+   * Encode an iterator on the input for the single input table associated with this job.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param cfg
+   *          the configuration of the iterator
+   * @throws IllegalArgumentException
+   *           if the iterator can't be serialized into the configuration
+   * @since 1.6.0
+   */
+  public static void addIterator(Class<?> implementingClass, Configuration conf, IteratorSetting cfg) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    String newIter;
+    try {
+      cfg.write(new DataOutputStream(baos));
+      newIter = new String(Base64.encodeBase64(baos.toByteArray()), StandardCharsets.UTF_8);
+      baos.close();
+    } catch (IOException e) {
+      throw new IllegalArgumentException("unable to serialize IteratorSetting");
+    }
+
+    String confKey = enumToConfKey(implementingClass, ScanOpts.ITERATORS);
+    String iterators = conf.get(confKey);
+    // No iterators specified yet, create a new string
+    if (iterators == null || iterators.isEmpty()) {
+      iterators = newIter;
+    } else {
+      // append the next iterator & reset
+      iterators = iterators.concat(StringUtils.COMMA_STR + newIter);
+    }
+    // Store the iterators w/ the job
+    conf.set(confKey, iterators);
+  }
+
+  /**
+   * 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 implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param enableFeature
+   *          the feature is enabled if true, disabled otherwise
+   * @see #setRanges(Class, Configuration, Collection)
+   * @since 1.6.0
+   */
+  public static void setAutoAdjustRanges(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+    conf.setBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), enableFeature);
+  }
+
+  /**
+   * Determines whether a configuration has auto-adjust ranges enabled.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return false if the feature is disabled, true otherwise
+   * @since 1.6.0
+   * @see #setAutoAdjustRanges(Class, Configuration, boolean)
+   */
+  public static Boolean getAutoAdjustRanges(Class<?> implementingClass, Configuration conf) {
+    return conf.getBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), true);
+  }
+
+  /**
+   * Controls the use of the {@link IsolatedScanner} in this job.
+   * 
+   * <p>
+   * By default, this feature is <b>disabled</b>.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param enableFeature
+   *          the feature is enabled if true, disabled otherwise
+   * @since 1.6.0
+   */
+  public static void setScanIsolation(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+    conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), enableFeature);
+  }
+
+  /**
+   * Determines whether a configuration has isolation enabled.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return true if the feature is enabled, false otherwise
+   * @since 1.6.0
+   * @see #setScanIsolation(Class, Configuration, boolean)
+   */
+  public static Boolean isIsolated(Class<?> implementingClass, Configuration conf) {
+    return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), false);
+  }
+
+  /**
+   * 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 implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param enableFeature
+   *          the feature is enabled if true, disabled otherwise
+   * @since 1.6.0
+   */
+  public static void setLocalIterators(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+    conf.setBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), enableFeature);
+  }
+
+  /**
+   * Determines whether a configuration uses local iterators.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return true if the feature is enabled, false otherwise
+   * @since 1.6.0
+   * @see #setLocalIterators(Class, Configuration, boolean)
+   */
+  public static Boolean usesLocalIterators(Class<?> implementingClass, Configuration conf) {
+    return conf.getBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), false);
+  }
+
+  /**
+   * <p>
+   * Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
+   * table's files. If the table is not offline, then the job will fail. If the table comes online during the map reduce job, it is likely that the job will
+   * fail.
+   * 
+   * <p>
+   * To use this option, the map reduce user will need access to read the Accumulo directory in HDFS.
+   * 
+   * <p>
+   * Reading the offline table will create the scan time iterator stack in the map process. So any iterators that are configured for the table will need to be
+   * on the mapper's classpath.
+   * 
+   * <p>
+   * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
+   * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
+   * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
+   * 
+   * <p>
+   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
+   * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
+   * 
+   * <p>
+   * By default, this feature is <b>disabled</b>.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param enableFeature
+   *          the feature is enabled if true, disabled otherwise
+   * @since 1.6.0
+   */
+  public static void setOfflineTableScan(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+    conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), enableFeature);
+  }
+
+  /**
+   * Determines whether a configuration has the offline table scan feature enabled.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return true if the feature is enabled, false otherwise
+   * @since 1.6.0
+   * @see #setOfflineTableScan(Class, Configuration, boolean)
+   */
+  public static Boolean isOfflineScan(Class<?> implementingClass, Configuration conf) {
+    return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), false);
+  }
+
+  /**
+   * Sets configurations for multiple tables at a time.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param configs
+   *          an array of {@link InputTableConfig} objects to associate with the job
+   * @since 1.6.0
+   */
+  public static void setInputTableConfigs(Class<?> implementingClass, Configuration conf, Map<String,InputTableConfig> configs) {
+    MapWritable mapWritable = new MapWritable();
+    for (Map.Entry<String,InputTableConfig> tableConfig : configs.entrySet())
+      mapWritable.put(new Text(tableConfig.getKey()), tableConfig.getValue());
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    try {
+      mapWritable.write(new DataOutputStream(baos));
+    } catch (IOException e) {
+      throw new IllegalStateException("Table configuration could not be serialized.");
+    }
+
+    String confKey = enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS);
+    conf.set(confKey, new String(Base64.encodeBase64(baos.toByteArray())));
+  }
+
+  /**
+   * Returns all {@link InputTableConfig} objects associated with this job.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return all of the table query configs for the job
+   * @since 1.6.0
+   */
+  public static Map<String,InputTableConfig> getInputTableConfigs(Class<?> implementingClass, Configuration conf) {
+    Map<String,InputTableConfig> configs = new HashMap<String,InputTableConfig>();
+    Map.Entry<String,InputTableConfig> defaultConfig = getDefaultInputTableConfig(implementingClass, conf);
+    if (defaultConfig != null)
+      configs.put(defaultConfig.getKey(), defaultConfig.getValue());
+    String configString = conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS));
+    MapWritable mapWritable = new MapWritable();
+    if (configString != null) {
+      try {
+        byte[] bytes = Base64.decodeBase64(configString.getBytes());
+        ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+        mapWritable.readFields(new DataInputStream(bais));
+        bais.close();
+      } catch (IOException e) {
+        throw new IllegalStateException("The table query configurations could not be deserialized from the given configuration");
+      }
+    }
+    for (Map.Entry<Writable,Writable> entry : mapWritable.entrySet())
+      configs.put(((Text) entry.getKey()).toString(), (InputTableConfig) entry.getValue());
+
+    return configs;
+  }
+
+  /**
+   * Returns the {@link InputTableConfig} for the given table
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param tableName
+   *          the table name for which to fetch the table query config
+   * @return the table query config for the given table name (if it exists) and null if it does not
+   * @since 1.6.0
+   */
+  public static InputTableConfig getInputTableConfig(Class<?> implementingClass, Configuration conf, String tableName) {
+    Map<String,InputTableConfig> queryConfigs = getInputTableConfigs(implementingClass, conf);
+    return queryConfigs.get(tableName);
+  }
+
+  /**
+   * Initializes an Accumulo {@link TabletLocator} based on the configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param tableId
+   *          The table id for which to initialize the {@link TabletLocator}
+   * @return an Accumulo tablet locator
+   * @throws TableNotFoundException
+   *           if the table name set on the configuration doesn't exist
+   * @since 1.6.0
+   */
+  public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf, String tableId) throws TableNotFoundException {
+    String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
+    if ("MockInstance".equals(instanceType))
+      return new MockTabletLocator();
+    Instance instance = getInstance(implementingClass, conf);
+    return TabletLocator.getLocator(instance, new Text(tableId));
+  }
+
+  // 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 implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @throws IOException
+   *           if the context is improperly configured
+   * @since 1.6.0
+   */
+  public static void validateOptions(Class<?> implementingClass, Configuration conf) throws IOException {
+
+    Map<String,InputTableConfig> inputTableConfigs = getInputTableConfigs(implementingClass, conf);
+    if (!isConnectorInfoSet(implementingClass, conf))
+      throw new IOException("Input info has not been set.");
+    String instanceKey = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
+    if (!"MockInstance".equals(instanceKey) && !"ZooKeeperInstance".equals(instanceKey))
+      throw new IOException("Instance info has not been set.");
+    // validate that we can connect as configured
+    try {
+      String principal = getPrincipal(implementingClass, conf);
+      AuthenticationToken token = getAuthenticationToken(implementingClass, conf);
+      Connector c = getInstance(implementingClass, conf).getConnector(principal, token);
+      if (!c.securityOperations().authenticateUser(principal, token))
+        throw new IOException("Unable to authenticate user");
+
+      if (getInputTableConfigs(implementingClass, conf).size() == 0)
+        throw new IOException("No table set.");
+
+      for (Map.Entry<String,InputTableConfig> tableConfig : inputTableConfigs.entrySet()) {
+        if (!c.securityOperations().hasTablePermission(getPrincipal(implementingClass, conf), tableConfig.getKey(), TablePermission.READ))
+          throw new IOException("Unable to access table");
+      }
+      for (Map.Entry<String,InputTableConfig> tableConfigEntry : inputTableConfigs.entrySet()) {
+        InputTableConfig tableConfig = tableConfigEntry.getValue();
+        if (!tableConfig.shouldUseLocalIterators()) {
+          if (tableConfig.getIterators() != null) {
+            for (IteratorSetting iter : tableConfig.getIterators()) {
+              if (!c.tableOperations().testClassLoad(tableConfigEntry.getKey(), iter.getIteratorClass(), SortedKeyValueIterator.class.getName()))
+                throw new AccumuloException("Servers are unable to load " + iter.getIteratorClass() + " as a " + SortedKeyValueIterator.class.getName());
+            }
+          }
+        }
+      }
+    } catch (AccumuloException e) {
+      throw new IOException(e);
+    } catch (AccumuloSecurityException e) {
+      throw new IOException(e);
+    } catch (TableNotFoundException e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Returns the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} for the configuration based on the properties set using the single-table
+   * input methods.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop instance for which to retrieve the configuration
+   * @return the config object built from the single input table properties set on the job
+   * @since 1.6.0
+   */
+  protected static Map.Entry<String,InputTableConfig> getDefaultInputTableConfig(Class<?> implementingClass, Configuration conf) {
+    String tableName = getInputTableName(implementingClass, conf);
+    if (tableName != null) {
+      InputTableConfig queryConfig = new InputTableConfig();
+      List<IteratorSetting> itrs = getIterators(implementingClass, conf);
+      if (itrs != null)
+        queryConfig.setIterators(itrs);
+      Set<Pair<Text,Text>> columns = getFetchedColumns(implementingClass, conf);
+      if (columns != null)
+        queryConfig.fetchColumns(columns);
+      List<Range> ranges = null;
+      try {
+        ranges = getRanges(implementingClass, conf);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      if (ranges != null)
+        queryConfig.setRanges(ranges);
+
+      queryConfig.setAutoAdjustRanges(getAutoAdjustRanges(implementingClass, conf)).setUseIsolatedScanners(isIsolated(implementingClass, conf))
+          .setUseLocalIterators(usesLocalIterators(implementingClass, conf)).setOfflineScan(isOfflineScan(implementingClass, conf));
+      return Maps.immutableEntry(tableName, queryConfig);
+    }
+    return null;
+  }
+
+  public static Map<String,Map<KeyExtent,List<Range>>> binOffline(String tableId, List<Range> ranges, Instance instance, Connector conn)
+      throws AccumuloException, TableNotFoundException {
+    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
+
+    if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
+      Tables.clearCache(instance);
+      if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
+        throw new AccumuloException("Table is online tableId:" + tableId + " cannot scan table in offline mode ");
+      }
+    }
+
+    for (Range range : ranges) {
+      Text startRow;
+
+      if (range.getStartKey() != null)
+        startRow = range.getStartKey().getRow();
+      else
+        startRow = new Text();
+
+      Range metadataRange = new Range(new KeyExtent(new Text(tableId), startRow, null).getMetadataEntry(), true, null, false);
+      Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+      MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
+      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME);
+      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
+      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME);
+      scanner.setRange(metadataRange);
+
+      RowIterator rowIter = new RowIterator(scanner);
+      KeyExtent lastExtent = null;
+      while (rowIter.hasNext()) {
+        Iterator<Map.Entry<Key,Value>> row = rowIter.next();
+        String last = "";
+        KeyExtent extent = null;
+        String location = null;
+
+        while (row.hasNext()) {
+          Map.Entry<Key,Value> entry = row.next();
+          Key key = entry.getKey();
+
+          if (key.getColumnFamily().equals(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME)) {
+            last = entry.getValue().toString();
+          }
+
+          if (key.getColumnFamily().equals(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME)
+              || key.getColumnFamily().equals(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME)) {
+            location = entry.getValue().toString();
+          }
+
+          if (MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
+            extent = new KeyExtent(key.getRow(), entry.getValue());
+          }
+
+        }
+
+        if (location != null)
+          return null;
+
+        if (!extent.getTableId().toString().equals(tableId)) {
+          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
+        }
+
+        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
+          throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
+        }
+
+        Map<KeyExtent,List<Range>> tabletRanges = binnedRanges.get(last);
+        if (tabletRanges == null) {
+          tabletRanges = new HashMap<KeyExtent,List<Range>>();
+          binnedRanges.put(last, tabletRanges);
+        }
+
+        List<Range> rangeList = tabletRanges.get(extent);
+        if (rangeList == null) {
+          rangeList = new ArrayList<Range>();
+          tabletRanges.put(extent, rangeList);
+        }
+
+        rangeList.add(range);
+
+        if (extent.getEndRow() == null || range.afterEndKey(new Key(extent.getEndRow()).followingKey(PartialKey.ROW))) {
+          break;
+        }
+
+        lastExtent = extent;
+      }
+
+    }
+    return binnedRanges;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4dfcb9de/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/OutputConfigurator.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/OutputConfigurator.java b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/OutputConfigurator.java
new file mode 100644
index 0000000..727971a
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/OutputConfigurator.java
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapreduce.lib.impl;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * @since 1.6.0
+ */
+public class OutputConfigurator extends ConfiguratorBase {
+
+  /**
+   * Configuration keys for {@link BatchWriter}.
+   * 
+   * @since 1.6.0
+   */
+  public static enum WriteOpts {
+    DEFAULT_TABLE_NAME, BATCH_WRITER_CONFIG
+  }
+
+  /**
+   * Configuration keys for various features.
+   * 
+   * @since 1.6.0
+   */
+  public static enum Features {
+    CAN_CREATE_TABLES, SIMULATION_MODE
+  }
+
+  /**
+   * 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 implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param tableName
+   *          the table to use when the tablename is null in the write call
+   * @since 1.6.0
+   */
+  public static void setDefaultTableName(Class<?> implementingClass, Configuration conf, String tableName) {
+    if (tableName != null)
+      conf.set(enumToConfKey(implementingClass, WriteOpts.DEFAULT_TABLE_NAME), tableName);
+  }
+
+  /**
+   * Gets the default table name from the configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return the default table name
+   * @since 1.6.0
+   * @see #setDefaultTableName(Class, Configuration, String)
+   */
+  public static String getDefaultTableName(Class<?> implementingClass, Configuration conf) {
+    return conf.get(enumToConfKey(implementingClass, WriteOpts.DEFAULT_TABLE_NAME));
+  }
+
+  /**
+   * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new {@link BatchWriterConfig}, with sensible built-in defaults is
+   * used. Setting the configuration multiple times overwrites any previous configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param bwConfig
+   *          the configuration for the {@link BatchWriter}
+   * @since 1.6.0
+   */
+  public static void setBatchWriterOptions(Class<?> implementingClass, Configuration conf, BatchWriterConfig bwConfig) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    String serialized;
+    try {
+      bwConfig.write(new DataOutputStream(baos));
+      serialized = new String(baos.toByteArray(), StandardCharsets.UTF_8);
+      baos.close();
+    } catch (IOException e) {
+      throw new IllegalArgumentException("unable to serialize " + BatchWriterConfig.class.getName());
+    }
+    conf.set(enumToConfKey(implementingClass, WriteOpts.BATCH_WRITER_CONFIG), serialized);
+  }
+
+  /**
+   * Gets the {@link BatchWriterConfig} settings.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return the configuration object
+   * @since 1.6.0
+   * @see #setBatchWriterOptions(Class, Configuration, BatchWriterConfig)
+   */
+  public static BatchWriterConfig getBatchWriterOptions(Class<?> implementingClass, Configuration conf) {
+    String serialized = conf.get(enumToConfKey(implementingClass, WriteOpts.BATCH_WRITER_CONFIG));
+    BatchWriterConfig bwConfig = new BatchWriterConfig();
+    if (serialized == null || serialized.isEmpty()) {
+      return bwConfig;
+    } else {
+      try {
+        ByteArrayInputStream bais = new ByteArrayInputStream(serialized.getBytes(StandardCharsets.UTF_8));
+        bwConfig.readFields(new DataInputStream(bais));
+        bais.close();
+        return bwConfig;
+      } catch (IOException e) {
+        throw new IllegalArgumentException("unable to serialize " + BatchWriterConfig.class.getName());
+      }
+    }
+  }
+
+  /**
+   * 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 implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param enableFeature
+   *          the feature is enabled if true, disabled otherwise
+   * @since 1.6.0
+   */
+  public static void setCreateTables(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+    conf.setBoolean(enumToConfKey(implementingClass, Features.CAN_CREATE_TABLES), enableFeature);
+  }
+
+  /**
+   * Determines whether tables are permitted to be created as needed.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return true if the feature is disabled, false otherwise
+   * @since 1.6.0
+   * @see #setCreateTables(Class, Configuration, boolean)
+   */
+  public static Boolean canCreateTables(Class<?> implementingClass, Configuration conf) {
+    return conf.getBoolean(enumToConfKey(implementingClass, Features.CAN_CREATE_TABLES), false);
+  }
+
+  /**
+   * 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 implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param enableFeature
+   *          the feature is enabled if true, disabled otherwise
+   * @since 1.6.0
+   */
+  public static void setSimulationMode(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+    conf.setBoolean(enumToConfKey(implementingClass, Features.SIMULATION_MODE), enableFeature);
+  }
+
+  /**
+   * Determines whether this feature is enabled.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return true if the feature is enabled, false otherwise
+   * @since 1.6.0
+   * @see #setSimulationMode(Class, Configuration, boolean)
+   */
+  public static Boolean getSimulationMode(Class<?> implementingClass, Configuration conf) {
+    return conf.getBoolean(enumToConfKey(implementingClass, Features.SIMULATION_MODE), false);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4dfcb9de/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/package-info.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/package-info.java b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/package-info.java
new file mode 100644
index 0000000..243160d
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/package-info.java
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+/**
+ * This package exists to store common helpers for configuring MapReduce jobs in a single location. It contains static configurator methods, stored in classes
+ * separate from the things they configure (typically, {@link org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat}/
+ * {@link org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat} and related classes in compatible frameworks), rather than storing them in those
+ * InputFormats/OutputFormats, so as not to clutter their API with methods that don't match the conventions for that framework. These classes may be useful to
+ * input/output plugins for other frameworks, so they can reuse the same configuration options and/or serialize them into a
+ * {@link org.apache.hadoop.conf.Configuration} instance in a standard way.
+ * 
+ * <p>
+ * It is not expected these will change much (except when new features are added), but end users should not use these classes. They should use the static
+ * configurators on the {@link org.apache.hadoop.mapreduce.InputFormat} or {@link org.apache.hadoop.mapreduce.OutputFormat} they are configuring, which in turn
+ * may use these classes to implement their own static configurators. Once again, these classes are intended for internal use, but may be useful to developers
+ * of plugins for other frameworks that read/write to Accumulo.
+ * 
+ * @since 1.6.0
+ */
+package org.apache.accumulo.core.client.mapreduce.lib.impl;
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4dfcb9de/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java
new file mode 100644
index 0000000..c59841d
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapreduce.lib.partition;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Partitioner;
+
+/**
+ * Hadoop partitioner that uses ranges based on row keys, and optionally sub-bins based on hashing.
+ */
+public class KeyRangePartitioner extends Partitioner<Key,Writable> implements Configurable {
+  private RangePartitioner rp = new RangePartitioner();
+  
+  @Override
+  public int getPartition(Key key, Writable value, int numPartitions) {
+    return rp.getPartition(key.getRow(), value, numPartitions);
+  }
+  
+  @Override
+  public Configuration getConf() {
+    return rp.getConf();
+  }
+  
+  @Override
+  public void setConf(Configuration conf) {
+    rp.setConf(conf);
+  }
+  
+  /**
+   * Sets the hdfs file name to use, containing a newline separated list of Base64 encoded split points that represent ranges for partitioning
+   */
+  public static void setSplitFile(Job job, String file) {
+    RangePartitioner.setSplitFile(job, file);
+  }
+  
+  /**
+   * Sets the number of random sub-bins per range
+   */
+  public static void setNumSubBins(Job job, int num) {
+    RangePartitioner.setNumSubBins(job, num);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4dfcb9de/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
new file mode 100644
index 0000000..1b7501c
--- /dev/null
+++ b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapreduce.lib.partition;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Scanner;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.mapreduce.lib.impl.DistributedCacheHelper;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Partitioner;
+
+/**
+ * Hadoop partitioner that uses ranges, and optionally sub-bins based on hashing.
+ */
+public class RangePartitioner extends Partitioner<Text,Writable> implements Configurable {
+  private static final String PREFIX = RangePartitioner.class.getName();
+  private static final String CUTFILE_KEY = PREFIX + ".cutFile";
+  private static final String NUM_SUBBINS = PREFIX + ".subBins";
+
+  private Configuration conf;
+
+  @Override
+  public int getPartition(Text key, Writable value, int numPartitions) {
+    try {
+      return findPartition(key, getCutPoints(), getNumSubBins());
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  int findPartition(Text key, Text[] array, int numSubBins) {
+    // find the bin for the range, and guarantee it is positive
+    int index = Arrays.binarySearch(array, key);
+    index = index < 0 ? (index + 1) * -1 : index;
+
+    // both conditions work with numSubBins == 1, but this check is to avoid
+    // hashing, when we don't need to, for speed
+    if (numSubBins < 2)
+      return index;
+    return (key.toString().hashCode() & Integer.MAX_VALUE) % numSubBins + index * numSubBins;
+  }
+
+  private int _numSubBins = 0;
+
+  private synchronized int getNumSubBins() {
+    if (_numSubBins < 1) {
+      // get number of sub-bins and guarantee it is positive
+      _numSubBins = Math.max(1, getConf().getInt(NUM_SUBBINS, 1));
+    }
+    return _numSubBins;
+  }
+
+  private Text cutPointArray[] = null;
+
+  private synchronized Text[] getCutPoints() throws IOException {
+    if (cutPointArray == null) {
+      String cutFileName = conf.get(CUTFILE_KEY);
+      Path[] cf = DistributedCacheHelper.getLocalCacheFiles(conf);
+
+      if (cf != null) {
+        for (Path path : cf) {
+          if (path.toUri().getPath().endsWith(cutFileName.substring(cutFileName.lastIndexOf('/')))) {
+            TreeSet<Text> cutPoints = new TreeSet<Text>();
+            Scanner in = new Scanner(new BufferedReader(new InputStreamReader(new FileInputStream(path.toString()), StandardCharsets.UTF_8)));
+            try {
+              while (in.hasNextLine())
+                cutPoints.add(new Text(Base64.decodeBase64(in.nextLine().getBytes(StandardCharsets.UTF_8))));
+            } finally {
+              in.close();
+            }
+            cutPointArray = cutPoints.toArray(new Text[cutPoints.size()]);
+            break;
+          }
+        }
+      }
+      if (cutPointArray == null)
+        throw new FileNotFoundException(cutFileName + " not found in distributed cache");
+    }
+    return cutPointArray;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Sets the hdfs file name to use, containing a newline separated list of Base64 encoded split points that represent ranges for partitioning
+   */
+  public static void setSplitFile(Job job, String file) {
+    URI uri = new Path(file).toUri();
+    DistributedCacheHelper.addCacheFile(uri, job.getConfiguration());
+    job.getConfiguration().set(CUTFILE_KEY, uri.getPath());
+  }
+
+  /**
+   * Sets the number of random sub-bins per range
+   */
+  public static void setNumSubBins(Job job, int num) {
+    job.getConfiguration().setInt(NUM_SUBBINS, num);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4dfcb9de/mapreduce/src/main/resources/.gitignore
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/resources/.gitignore b/mapreduce/src/main/resources/.gitignore
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4dfcb9de/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java b/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
new file mode 100644
index 0000000..aad544b
--- /dev/null
+++ b/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
@@ -0,0 +1,247 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapred;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Logger;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class AccumuloFileOutputFormatTest {
+  private static final String PREFIX = AccumuloFileOutputFormatTest.class.getSimpleName();
+  private static final String INSTANCE_NAME = PREFIX + "_mapred_instance";
+  private static final String BAD_TABLE = PREFIX + "_mapred_bad_table";
+  private static final String TEST_TABLE = PREFIX + "_mapred_test_table";
+  private static final String EMPTY_TABLE = PREFIX + "_mapred_empty_table";
+
+  private static AssertionError e1 = null;
+  private static AssertionError e2 = null;
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
+    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
+    c.tableOperations().create(EMPTY_TABLE);
+    c.tableOperations().create(TEST_TABLE);
+    c.tableOperations().create(BAD_TABLE);
+    BatchWriter bw = c.createBatchWriter(TEST_TABLE, new BatchWriterConfig());
+    Mutation m = new Mutation("Key");
+    m.put("", "", "");
+    bw.addMutation(m);
+    bw.close();
+    bw = c.createBatchWriter(BAD_TABLE, new BatchWriterConfig());
+    m = new Mutation("r1");
+    m.put("cf1", "cq1", "A&B");
+    m.put("cf1", "cq1", "A&B");
+    m.put("cf1", "cq2", "A&");
+    bw.addMutation(m);
+    bw.close();
+  }
+
+  @Test
+  public void testEmptyWrite() throws Exception {
+    handleWriteTests(false);
+  }
+
+  @Test
+  public void testRealWrite() throws Exception {
+    handleWriteTests(true);
+  }
+
+  private static class MRTester extends Configured implements Tool {
+    private static class BadKeyMapper implements Mapper<Key,Value,Key,Value> {
+
+      int index = 0;
+
+      @Override
+      public void map(Key key, Value value, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
+        try {
+          try {
+            output.collect(key, value);
+            if (index == 2)
+              fail();
+          } catch (Exception e) {
+            Logger.getLogger(this.getClass()).error(e, e);
+            assertEquals(2, index);
+          }
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        index++;
+      }
+
+      @Override
+      public void configure(JobConf job) {}
+
+      @Override
+      public void close() throws IOException {
+        try {
+          assertEquals(2, index);
+        } catch (AssertionError e) {
+          e2 = e;
+        }
+      }
+
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 4) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table> <outputfile>");
+      }
+
+      String user = args[0];
+      String pass = args[1];
+      String table = args[2];
+
+      JobConf job = new JobConf(getConf());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormat(AccumuloInputFormat.class);
+
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
+      AccumuloInputFormat.setInputTableName(job, table);
+      AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
+      AccumuloFileOutputFormat.setOutputPath(job, new Path(args[3]));
+
+      job.setMapperClass(BAD_TABLE.equals(table) ? BadKeyMapper.class : IdentityMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormat(AccumuloFileOutputFormat.class);
+
+      job.setNumReduceTasks(0);
+
+      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      assertEquals(0, ToolRunner.run(CachedConfiguration.getInstance(), new MRTester(), args));
+    }
+  }
+
+  public void handleWriteTests(boolean content) throws Exception {
+    File f = folder.newFile("handleWriteTests");
+    f.delete();
+    MRTester.main(new String[] {"root", "", content ? TEST_TABLE : EMPTY_TABLE, f.getAbsolutePath()});
+
+    assertTrue(f.exists());
+    File[] files = f.listFiles(new FileFilter() {
+      @Override
+      public boolean accept(File file) {
+        return file.getName().startsWith("part-m-");
+      }
+    });
+    if (content) {
+      assertEquals(1, files.length);
+      assertTrue(files[0].exists());
+    } else {
+      assertEquals(0, files.length);
+    }
+  }
+
+  @Test
+  public void writeBadVisibility() throws Exception {
+    File f = folder.newFile("writeBadVisibility");
+    f.delete();
+    MRTester.main(new String[] {"root", "", BAD_TABLE, f.getAbsolutePath()});
+    Logger.getLogger(this.getClass()).error(e1, e1);
+    assertNull(e1);
+    assertNull(e2);
+  }
+
+  @Test
+  public void validateConfiguration() throws IOException, InterruptedException {
+
+    int a = 7;
+    long b = 300l;
+    long c = 50l;
+    long d = 10l;
+    String e = "snappy";
+
+    JobConf job = new JobConf();
+    AccumuloFileOutputFormat.setReplication(job, a);
+    AccumuloFileOutputFormat.setFileBlockSize(job, b);
+    AccumuloFileOutputFormat.setDataBlockSize(job, c);
+    AccumuloFileOutputFormat.setIndexBlockSize(job, d);
+    AccumuloFileOutputFormat.setCompressionType(job, e);
+
+    AccumuloConfiguration acuconf = AccumuloFileOutputFormat.getAccumuloConfiguration(job);
+
+    assertEquals(7, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
+    assertEquals(300l, acuconf.getMemoryInBytes(Property.TABLE_FILE_BLOCK_SIZE));
+    assertEquals(50l, acuconf.getMemoryInBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE));
+    assertEquals(10l, acuconf.getMemoryInBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX));
+    assertEquals("snappy", acuconf.get(Property.TABLE_FILE_COMPRESSION_TYPE));
+
+    a = 17;
+    b = 1300l;
+    c = 150l;
+    d = 110l;
+    e = "lzo";
+
+    job = new JobConf();
+    AccumuloFileOutputFormat.setReplication(job, a);
+    AccumuloFileOutputFormat.setFileBlockSize(job, b);
+    AccumuloFileOutputFormat.setDataBlockSize(job, c);
+    AccumuloFileOutputFormat.setIndexBlockSize(job, d);
+    AccumuloFileOutputFormat.setCompressionType(job, e);
+
+    acuconf = AccumuloFileOutputFormat.getAccumuloConfiguration(job);
+
+    assertEquals(17, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
+    assertEquals(1300l, acuconf.getMemoryInBytes(Property.TABLE_FILE_BLOCK_SIZE));
+    assertEquals(150l, acuconf.getMemoryInBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE));
+    assertEquals(110l, acuconf.getMemoryInBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX));
+    assertEquals("lzo", acuconf.get(Property.TABLE_FILE_COMPRESSION_TYPE));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4dfcb9de/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java b/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
new file mode 100644
index 0000000..13490e0
--- /dev/null
+++ b/mapreduce/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapred;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.user.RegExFilter;
+import org.apache.accumulo.core.iterators.user.WholeRowIterator;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.lib.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class AccumuloInputFormatTest {
+
+  private static final String PREFIX = AccumuloInputFormatTest.class.getSimpleName();
+  private static final String INSTANCE_NAME = PREFIX + "_mapred_instance";
+  private static final String TEST_TABLE_1 = PREFIX + "_mapred_table_1";
+
+  private JobConf job;
+
+  @BeforeClass
+  public static void setupClass() {
+    System.setProperty("hadoop.tmp.dir", System.getProperty("user.dir") + "/target/hadoop-tmp");
+  }
+
+  @Before
+  public void createJob() {
+    job = new JobConf();
+  }
+
+  /**
+   * Check that the iterator configuration is getting stored in the Job conf correctly.
+   */
+  @Test
+  public void testSetIterator() throws IOException {
+    IteratorSetting is = new IteratorSetting(1, "WholeRow", "org.apache.accumulo.core.iterators.WholeRowIterator");
+    AccumuloInputFormat.addIterator(job, is);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    is.write(new DataOutputStream(baos));
+    String iterators = job.get("AccumuloInputFormat.ScanOpts.Iterators");
+    assertEquals(new String(Base64.encodeBase64(baos.toByteArray())), iterators);
+  }
+
+  @Test
+  public void testAddIterator() throws IOException {
+    AccumuloInputFormat.addIterator(job, new IteratorSetting(1, "WholeRow", WholeRowIterator.class));
+    AccumuloInputFormat.addIterator(job, new IteratorSetting(2, "Versions", "org.apache.accumulo.core.iterators.VersioningIterator"));
+    IteratorSetting iter = new IteratorSetting(3, "Count", "org.apache.accumulo.core.iterators.CountingIterator");
+    iter.addOption("v1", "1");
+    iter.addOption("junk", "\0omg:!\\xyzzy");
+    AccumuloInputFormat.addIterator(job, iter);
+
+    List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
+
+    // Check the list size
+    assertTrue(list.size() == 3);
+
+    // Walk the list and make sure our settings are correct
+    IteratorSetting setting = list.get(0);
+    assertEquals(1, setting.getPriority());
+    assertEquals("org.apache.accumulo.core.iterators.user.WholeRowIterator", setting.getIteratorClass());
+    assertEquals("WholeRow", setting.getName());
+    assertEquals(0, setting.getOptions().size());
+
+    setting = list.get(1);
+    assertEquals(2, setting.getPriority());
+    assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
+    assertEquals("Versions", setting.getName());
+    assertEquals(0, setting.getOptions().size());
+
+    setting = list.get(2);
+    assertEquals(3, setting.getPriority());
+    assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
+    assertEquals("Count", setting.getName());
+    assertEquals(2, setting.getOptions().size());
+    assertEquals("1", setting.getOptions().get("v1"));
+    assertEquals("\0omg:!\\xyzzy", setting.getOptions().get("junk"));
+  }
+
+  /**
+   * Test adding iterator options where the keys and values contain both the FIELD_SEPARATOR character (':') and ITERATOR_SEPARATOR (',') characters. There
+   * should be no exceptions thrown when trying to parse these types of option entries.
+   * 
+   * This test makes sure that the expected raw values, as appears in the Job, are equal to what's expected.
+   */
+  @Test
+  public void testIteratorOptionEncoding() throws Throwable {
+    String key = "colon:delimited:key";
+    String value = "comma,delimited,value";
+    IteratorSetting someSetting = new IteratorSetting(1, "iterator", "Iterator.class");
+    someSetting.addOption(key, value);
+    AccumuloInputFormat.addIterator(job, someSetting);
+
+    List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
+    assertEquals(1, list.size());
+    assertEquals(1, list.get(0).getOptions().size());
+    assertEquals(list.get(0).getOptions().get(key), value);
+
+    someSetting.addOption(key + "2", value);
+    someSetting.setPriority(2);
+    someSetting.setName("it2");
+    AccumuloInputFormat.addIterator(job, someSetting);
+    list = AccumuloInputFormat.getIterators(job);
+    assertEquals(2, list.size());
+    assertEquals(1, list.get(0).getOptions().size());
+    assertEquals(list.get(0).getOptions().get(key), value);
+    assertEquals(2, list.get(1).getOptions().size());
+    assertEquals(list.get(1).getOptions().get(key), value);
+    assertEquals(list.get(1).getOptions().get(key + "2"), value);
+  }
+
+  /**
+   * Test getting iterator settings for multiple iterators set
+   */
+  @Test
+  public void testGetIteratorSettings() throws IOException {
+    AccumuloInputFormat.addIterator(job, new IteratorSetting(1, "WholeRow", "org.apache.accumulo.core.iterators.WholeRowIterator"));
+    AccumuloInputFormat.addIterator(job, new IteratorSetting(2, "Versions", "org.apache.accumulo.core.iterators.VersioningIterator"));
+    AccumuloInputFormat.addIterator(job, new IteratorSetting(3, "Count", "org.apache.accumulo.core.iterators.CountingIterator"));
+
+    List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
+
+    // Check the list size
+    assertTrue(list.size() == 3);
+
+    // Walk the list and make sure our settings are correct
+    IteratorSetting setting = list.get(0);
+    assertEquals(1, setting.getPriority());
+    assertEquals("org.apache.accumulo.core.iterators.WholeRowIterator", setting.getIteratorClass());
+    assertEquals("WholeRow", setting.getName());
+
+    setting = list.get(1);
+    assertEquals(2, setting.getPriority());
+    assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
+    assertEquals("Versions", setting.getName());
+
+    setting = list.get(2);
+    assertEquals(3, setting.getPriority());
+    assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
+    assertEquals("Count", setting.getName());
+
+  }
+
+  @Test
+  public void testSetRegex() throws IOException {
+    String regex = ">\"*%<>\'\\";
+
+    IteratorSetting is = new IteratorSetting(50, regex, RegExFilter.class);
+    RegExFilter.setRegexs(is, regex, null, null, null, false);
+    AccumuloInputFormat.addIterator(job, is);
+
+    assertTrue(regex.equals(AccumuloInputFormat.getIterators(job).get(0).getName()));
+  }
+
+  private static AssertionError e1 = null;
+  private static AssertionError e2 = null;
+
+  private static class MRTester extends Configured implements Tool {
+    private static class TestMapper implements Mapper<Key,Value,Key,Value> {
+      Key key = null;
+      int count = 0;
+
+      @Override
+      public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
+        try {
+          if (key != null)
+            assertEquals(key.getRow().toString(), new String(v.get()));
+          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
+          assertEquals(new String(v.get()), String.format("%09x", count));
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        key = new Key(k);
+        count++;
+      }
+
+      @Override
+      public void configure(JobConf job) {}
+
+      @Override
+      public void close() throws IOException {
+        try {
+          assertEquals(100, count);
+        } catch (AssertionError e) {
+          e2 = e;
+        }
+      }
+
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 3) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table>");
+      }
+
+      String user = args[0];
+      String pass = args[1];
+      String table = args[2];
+
+      JobConf job = new JobConf(getConf());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormat(AccumuloInputFormat.class);
+
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
+      AccumuloInputFormat.setInputTableName(job, table);
+      AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormat(NullOutputFormat.class);
+
+      job.setNumReduceTasks(0);
+
+      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String... args) throws Exception {
+      assertEquals(0, ToolRunner.run(new Configuration(), new MRTester(), args));
+    }
+  }
+
+  @Test
+  public void testMap() throws Exception {
+    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
+    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
+    c.tableOperations().create(TEST_TABLE_1);
+    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    MRTester.main("root", "", TEST_TABLE_1);
+    assertNull(e1);
+    assertNull(e2);
+  }
+}