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

[06/11] Revert "ACCUMULO-1880 create mapreduce module"

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
new file mode 100644
index 0000000..80e8c28
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapreduce;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class RangeInputSplitTest {
+
+  @Test
+  public void testSimpleWritable() throws IOException {
+    RangeInputSplit split = new RangeInputSplit("table", "1", new Range(new Key("a"), new Key("b")), new String[]{"localhost"});
+    
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    split.write(dos);
+    
+    RangeInputSplit newSplit = new RangeInputSplit();
+    
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInputStream dis = new DataInputStream(bais);
+    newSplit.readFields(dis);
+    
+    Assert.assertEquals(split.getTableName(), newSplit.getTableName());
+    Assert.assertEquals(split.getTableId(), newSplit.getTableId());
+    Assert.assertEquals(split.getRange(), newSplit.getRange());
+    Assert.assertTrue(Arrays.equals(split.getLocations(), newSplit.getLocations()));
+  }
+
+  @Test
+  public void testAllFieldsWritable() throws IOException {
+    RangeInputSplit split = new RangeInputSplit("table", "1", new Range(new Key("a"), new Key("b")), new String[]{"localhost"});
+    
+    Set<Pair<Text,Text>> fetchedColumns = new HashSet<Pair<Text,Text>>();
+    
+    fetchedColumns.add(new Pair<Text,Text>(new Text("colf1"), new Text("colq1")));
+    fetchedColumns.add(new Pair<Text,Text>(new Text("colf2"), new Text("colq2")));
+    
+    split.setAuths(new Authorizations("foo"));
+    split.setOffline(true);
+    split.setIsolatedScan(true);
+    split.setUsesLocalIterators(true);
+    split.setFetchedColumns(fetchedColumns);
+    split.setToken(new PasswordToken("password"));
+    split.setPrincipal("root");
+    split.setInstanceName("instance");
+    split.setMockInstance(true);
+    split.setZooKeepers("localhost");
+    split.setLogLevel(Level.WARN);
+    
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    split.write(dos);
+    
+    RangeInputSplit newSplit = new RangeInputSplit();
+    
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInputStream dis = new DataInputStream(bais);
+    newSplit.readFields(dis);
+    
+    Assert.assertEquals(split.getRange(), newSplit.getRange());
+    Assert.assertArrayEquals(split.getLocations(), newSplit.getLocations());
+    
+    Assert.assertEquals(split.getAuths(), newSplit.getAuths());
+    Assert.assertEquals(split.isOffline(), newSplit.isOffline());
+    Assert.assertEquals(split.isIsolatedScan(), newSplit.isOffline());
+    Assert.assertEquals(split.usesLocalIterators(), newSplit.usesLocalIterators());
+    Assert.assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
+    Assert.assertEquals(split.getToken(), newSplit.getToken());
+    Assert.assertEquals(split.getPrincipal(), newSplit.getPrincipal());
+    Assert.assertEquals(split.getInstanceName(), newSplit.getInstanceName());
+    Assert.assertEquals(split.isMockInstance(), newSplit.isMockInstance());
+    Assert.assertEquals(split.getZooKeepers(), newSplit.getZooKeepers());
+    Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel());
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
new file mode 100644
index 0000000..fd207a1
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapreduce;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+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.Scanner;
+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.security.Authorizations;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * 
+ */
+public class TokenFileTest {
+  private static AssertionError e1 = null;
+  private static final String PREFIX = TokenFileTest.class.getSimpleName();
+  private static final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
+  private static final String TEST_TABLE_1 = PREFIX + "_mapreduce_table_1";
+  private static final String TEST_TABLE_2 = PREFIX + "_mapreduce_table_2";
+
+  private static class MRTokenFileTester extends Configured implements Tool {
+    private static class TestMapper extends Mapper<Key,Value,Text,Mutation> {
+      Key key = null;
+      int count = 0;
+
+      @Override
+      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
+        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
+      protected void cleanup(Context context) throws IOException, InterruptedException {
+        Mutation m = new Mutation("total");
+        m.put("", "", Integer.toString(count));
+        context.write(new Text(), m);
+      }
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 4) {
+        throw new IllegalArgumentException("Usage : " + MRTokenFileTester.class.getName() + " <user> <token file> <inputtable> <outputtable>");
+      }
+
+      String user = args[0];
+      String tokenFile = args[1];
+      String table1 = args[2];
+      String table2 = args[3];
+
+      @SuppressWarnings("deprecation")
+      Job job = new Job(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormatClass(AccumuloInputFormat.class);
+
+      AccumuloInputFormat.setConnectorInfo(job, user, tokenFile);
+      AccumuloInputFormat.setInputTableName(job, table1);
+      AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormatClass(AccumuloOutputFormat.class);
+      job.setOutputKeyClass(Text.class);
+      job.setOutputValueClass(Mutation.class);
+
+      AccumuloOutputFormat.setConnectorInfo(job, user, tokenFile);
+      AccumuloOutputFormat.setCreateTables(job, false);
+      AccumuloOutputFormat.setDefaultTableName(job, table2);
+      AccumuloOutputFormat.setMockInstance(job, INSTANCE_NAME);
+
+      job.setNumReduceTasks(0);
+
+      job.waitForCompletion(true);
+
+      return job.isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      Configuration conf = CachedConfiguration.getInstance();
+      conf.set("hadoop.tmp.dir", new File(args[1]).getParent());
+      assertEquals(0, ToolRunner.run(conf, new MRTokenFileTester(), args));
+    }
+  }
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
+
+  @Test
+  public void testMR() throws Exception {
+    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
+    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
+    c.tableOperations().create(TEST_TABLE_1);
+    c.tableOperations().create(TEST_TABLE_2);
+    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
+    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();
+
+    File tf = folder.newFile("root_test.pw");
+    PrintStream out = new PrintStream(tf);
+    String outString = new Credentials("root", new PasswordToken("")).serialize();
+    out.println(outString);
+    out.close();
+
+    MRTokenFileTester.main(new String[] {"root", tf.getAbsolutePath(), TEST_TABLE_1, TEST_TABLE_2});
+    assertNull(e1);
+
+    Scanner scanner = c.createScanner(TEST_TABLE_2, new Authorizations());
+    Iterator<Entry<Key,Value>> iter = scanner.iterator();
+    assertTrue(iter.hasNext());
+    Entry<Key,Value> entry = iter.next();
+    assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
+    assertFalse(iter.hasNext());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
new file mode 100644
index 0000000..1983470
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
@@ -0,0 +1,129 @@
+/*
+ * 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 org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class ConfiguratorBaseTest {
+
+  private static enum PrivateTestingEnum {
+    SOMETHING, SOMETHING_ELSE
+  }
+
+  @Test
+  public void testEnumToConfKey() {
+    assertEquals(this.getClass().getSimpleName() + ".PrivateTestingEnum.Something",
+        ConfiguratorBase.enumToConfKey(this.getClass(), PrivateTestingEnum.SOMETHING));
+    assertEquals(this.getClass().getSimpleName() + ".PrivateTestingEnum.SomethingElse",
+        ConfiguratorBase.enumToConfKey(this.getClass(), PrivateTestingEnum.SOMETHING_ELSE));
+  }
+
+  @Test
+  public void testSetConnectorInfoClassOfQConfigurationStringAuthenticationToken() throws AccumuloSecurityException {
+    Configuration conf = new Configuration();
+    assertFalse(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
+    ConfiguratorBase.setConnectorInfo(this.getClass(), conf, "testUser", new PasswordToken("testPassword"));
+    assertTrue(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
+    assertEquals("testUser", ConfiguratorBase.getPrincipal(this.getClass(), conf));
+    AuthenticationToken token = ConfiguratorBase.getAuthenticationToken(this.getClass(), conf);
+    assertEquals(PasswordToken.class, token.getClass());
+    assertEquals(new PasswordToken("testPassword"), token);
+    assertEquals(
+        "inline:" + PasswordToken.class.getName() + ":" + Base64.encodeBase64String(AuthenticationTokenSerializer.serialize(new PasswordToken("testPassword"))),
+        conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.ConnectorInfo.TOKEN)));
+  }
+
+  @Test
+  public void testSetConnectorInfoClassOfQConfigurationStringString() throws AccumuloSecurityException {
+    Configuration conf = new Configuration();
+    assertFalse(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
+    ConfiguratorBase.setConnectorInfo(this.getClass(), conf, "testUser", "testFile");
+    assertTrue(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
+    assertEquals("testUser", ConfiguratorBase.getPrincipal(this.getClass(), conf));
+    assertEquals("file:testFile", conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.ConnectorInfo.TOKEN)));
+  }
+
+  @Test
+  public void testSetZooKeeperInstance() {
+    Configuration conf = new Configuration();
+    ConfiguratorBase.setZooKeeperInstance(this.getClass(), conf, new ClientConfiguration().withInstance("testInstanceName").withZkHosts("testZooKeepers")
+        .withSsl(true).withZkTimeout(1234));
+    ClientConfiguration clientConf = ClientConfiguration.deserialize(conf.get(ConfiguratorBase.enumToConfKey(this.getClass(),
+        ConfiguratorBase.InstanceOpts.CLIENT_CONFIG)));
+    assertEquals("testInstanceName", clientConf.get(ClientProperty.INSTANCE_NAME));
+    assertEquals("testZooKeepers", clientConf.get(ClientProperty.INSTANCE_ZK_HOST));
+    assertEquals("true", clientConf.get(ClientProperty.INSTANCE_RPC_SSL_ENABLED));
+    assertEquals("1234", clientConf.get(ClientProperty.INSTANCE_ZK_TIMEOUT));
+    assertEquals(ZooKeeperInstance.class.getSimpleName(), conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.TYPE)));
+
+    Instance instance = ConfiguratorBase.getInstance(this.getClass(), conf);
+    assertEquals(ZooKeeperInstance.class.getName(), instance.getClass().getName());
+    assertEquals("testInstanceName", ((ZooKeeperInstance) instance).getInstanceName());
+    assertEquals("testZooKeepers", ((ZooKeeperInstance) instance).getZooKeepers());
+    assertEquals(1234000, ((ZooKeeperInstance) instance).getZooKeepersSessionTimeOut());
+  }
+
+  @Test
+  public void testSetMockInstance() {
+    Configuration conf = new Configuration();
+    ConfiguratorBase.setMockInstance(this.getClass(), conf, "testInstanceName");
+    assertEquals("testInstanceName", conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.NAME)));
+    assertEquals(null, conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.ZOO_KEEPERS)));
+    assertEquals(MockInstance.class.getSimpleName(), conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.TYPE)));
+    Instance instance = ConfiguratorBase.getInstance(this.getClass(), conf);
+    assertEquals(MockInstance.class.getName(), instance.getClass().getName());
+  }
+
+  @Test
+  public void testSetLogLevel() {
+    Configuration conf = new Configuration();
+    Level currentLevel = Logger.getLogger(this.getClass()).getLevel();
+
+    ConfiguratorBase.setLogLevel(this.getClass(), conf, Level.DEBUG);
+    Logger.getLogger(this.getClass()).setLevel(currentLevel);
+    assertEquals(Level.DEBUG, ConfiguratorBase.getLogLevel(this.getClass(), conf));
+
+    ConfiguratorBase.setLogLevel(this.getClass(), conf, Level.INFO);
+    Logger.getLogger(this.getClass()).setLevel(currentLevel);
+    assertEquals(Level.INFO, ConfiguratorBase.getLogLevel(this.getClass(), conf));
+
+    ConfiguratorBase.setLogLevel(this.getClass(), conf, Level.FATAL);
+    Logger.getLogger(this.getClass()).setLevel(currentLevel);
+    assertEquals(Level.FATAL, ConfiguratorBase.getLogLevel(this.getClass(), conf));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java
new file mode 100644
index 0000000..8fca169
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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 static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.junit.Test;
+
+public class RangePartitionerTest {
+  
+  private static Text[] cutArray = new Text[] {new Text("A"), new Text("B"), new Text("C")};
+  
+  @Test
+  public void testNoSubBins() throws IOException {
+    for (int i = -2; i < 2; ++i) {
+      checkExpectedBins(i, new String[] {"A", "B", "C"}, new int[] {0, 1, 2});
+      checkExpectedBins(i, new String[] {"C", "A", "B"}, new int[] {2, 0, 1});
+      checkExpectedBins(i, new String[] {"", "AA", "BB", "CC"}, new int[] {0, 1, 2, 3});
+    }
+  }
+  
+  @Test
+  public void testSubBins() throws IOException {
+    checkExpectedRangeBins(2, new String[] {"A", "B", "C"}, new int[] {1, 3, 5});
+    checkExpectedRangeBins(2, new String[] {"C", "A", "B"}, new int[] {5, 1, 3});
+    checkExpectedRangeBins(2, new String[] {"", "AA", "BB", "CC"}, new int[] {1, 3, 5, 7});
+    
+    checkExpectedRangeBins(3, new String[] {"A", "B", "C"}, new int[] {2, 5, 8});
+    checkExpectedRangeBins(3, new String[] {"C", "A", "B"}, new int[] {8, 2, 5});
+    checkExpectedRangeBins(3, new String[] {"", "AA", "BB", "CC"}, new int[] {2, 5, 8, 11});
+    
+    checkExpectedRangeBins(10, new String[] {"A", "B", "C"}, new int[] {9, 19, 29});
+    checkExpectedRangeBins(10, new String[] {"C", "A", "B"}, new int[] {29, 9, 19});
+    checkExpectedRangeBins(10, new String[] {"", "AA", "BB", "CC"}, new int[] {9, 19, 29, 39});
+  }
+  
+  private RangePartitioner prepPartitioner(int numSubBins) throws IOException {
+    @SuppressWarnings("deprecation")
+    Job job = new Job();
+    RangePartitioner.setNumSubBins(job, numSubBins);
+    RangePartitioner rp = new RangePartitioner();
+    rp.setConf(job.getConfiguration());
+    return rp;
+  }
+  
+  private void checkExpectedRangeBins(int numSubBins, String[] strings, int[] rangeEnds) throws IOException {
+    assertTrue(strings.length == rangeEnds.length);
+    for (int i = 0; i < strings.length; ++i) {
+      int endRange = rangeEnds[i];
+      int startRange = endRange + 1 - numSubBins;
+      int part = prepPartitioner(numSubBins).findPartition(new Text(strings[i]), cutArray, numSubBins);
+      assertTrue(part >= startRange);
+      assertTrue(part <= endRange);
+    }
+  }
+  
+  private void checkExpectedBins(int numSubBins, String[] strings, int[] bins) throws IOException {
+    assertTrue(strings.length == bins.length);
+    for (int i = 0; i < strings.length; ++i) {
+      int bin = bins[i], part = prepPartitioner(numSubBins).findPartition(new Text(strings[i]), cutArray, numSubBins);
+      assertTrue(bin == part);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/examples/simple/pom.xml
----------------------------------------------------------------------
diff --git a/examples/simple/pom.xml b/examples/simple/pom.xml
index 37adc00..8390d01 100644
--- a/examples/simple/pom.xml
+++ b/examples/simple/pom.xml
@@ -61,10 +61,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
-      <artifactId>accumulo-mapreduce</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-shell</artifactId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/mapreduce/pom.xml b/mapreduce/pom.xml
deleted file mode 100644
index c9cb110..0000000
--- a/mapreduce/pom.xml
+++ /dev/null
@@ -1,67 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.accumulo</groupId>
-    <artifactId>accumulo-project</artifactId>
-    <version>1.7.0-SNAPSHOT</version>
-  </parent>
-  <artifactId>accumulo-mapreduce</artifactId>
-  <name>MapReduce</name>
-  <description>The Map Reduce bindings for accessing Apache Accumulo.</description>
-  <dependencies>
-    <dependency>
-      <groupId>com.beust</groupId>
-      <artifactId>jcommander</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-codec</groupId>
-      <artifactId>commons-codec</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-collections</groupId>
-      <artifactId>commons-collections</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-configuration</groupId>
-      <artifactId>commons-configuration</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>log4j</groupId>
-      <artifactId>log4j</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.accumulo</groupId>
-      <artifactId>accumulo-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOnDefaultTable.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOnDefaultTable.java b/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOnDefaultTable.java
deleted file mode 100644
index e7a3dd4..0000000
--- a/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOnDefaultTable.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.cli;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.hadoop.mapreduce.Job;
-
-import com.beust.jcommander.Parameter;
-
-public class MapReduceClientOnDefaultTable extends MapReduceClientOpts {
-  @Parameter(names = "--table", description = "table to use")
-  public String tableName;
-
-  public MapReduceClientOnDefaultTable(String table) {
-    this.tableName = table;
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  @Override
-  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
-    super.setAccumuloConfigs(job);
-    AccumuloInputFormat.setConnectorInfo(job, principal, getToken());
-    AccumuloInputFormat.setInputTableName(job, getTableName());
-    AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloOutputFormat.setConnectorInfo(job, principal, getToken());
-    AccumuloOutputFormat.setCreateTables(job, true);
-    AccumuloOutputFormat.setDefaultTableName(job, getTableName());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOnRequiredTable.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOnRequiredTable.java b/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOnRequiredTable.java
deleted file mode 100644
index abfc17d..0000000
--- a/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOnRequiredTable.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.cli;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.hadoop.mapreduce.Job;
-
-import com.beust.jcommander.Parameter;
-
-public class MapReduceClientOnRequiredTable extends MapReduceClientOpts {
-  @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
-  private String tableName;
-
-  @Parameter(names = {"-tf", "--tokenFile"}, description = "File in hdfs containing the user's authentication token create with \"bin/accumulo create-token\"")
-  private String tokenFile = "";
-
-  @Override
-  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
-    super.setAccumuloConfigs(job);
-
-    if (tokenFile.isEmpty()) {
-      AccumuloInputFormat.setConnectorInfo(job, principal, getToken());
-      AccumuloOutputFormat.setConnectorInfo(job, principal, getToken());
-    } else {
-      AccumuloInputFormat.setConnectorInfo(job, principal, tokenFile);
-      AccumuloOutputFormat.setConnectorInfo(job, principal, tokenFile);
-    }
-    AccumuloInputFormat.setInputTableName(job, getTableName());
-    AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloOutputFormat.setCreateTables(job, true);
-    AccumuloOutputFormat.setDefaultTableName(job, getTableName());
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOpts.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOpts.java b/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOpts.java
deleted file mode 100644
index 4b3b7ed..0000000
--- a/mapreduce/src/main/java/org/apache/accumulo/core/cli/MapReduceClientOpts.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.cli;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.hadoop.mapreduce.Job;
-
-/**
- * Adds some MR awareness to the ClientOpts
- */
-public class MapReduceClientOpts extends ClientOpts {
-  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
-    AccumuloInputFormat.setZooKeeperInstance(job, this.getClientConfiguration());
-    AccumuloOutputFormat.setZooKeeperInstance(job, this.getClientConfiguration());
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
deleted file mode 100644
index 5af78d2..0000000
--- a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ /dev/null
@@ -1,607 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableDeletedException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.TableOfflineException;
-import org.apache.accumulo.core.client.impl.OfflineScanner;
-import org.apache.accumulo.core.client.impl.ScannerImpl;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
-import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * An abstract input format to provide shared methods common to all other input format classes. At the very least, any classes inheriting from this class will
- * need to define their own {@link RecordReader}.
- */
-public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
-  protected static final Class<?> CLASS = AccumuloInputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   * 
-   * <p>
-   * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to provide a charset safe
-   * conversion to a string, and is not intended to be secure.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param principal
-   *          a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param token
-   *          the user's password
-   * @since 1.5.0
-   */
-  public static void setConnectorInfo(JobConf job, String principal, AuthenticationToken token) throws AccumuloSecurityException {
-    InputConfigurator.setConnectorInfo(CLASS, job, principal, token);
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   * 
-   * <p>
-   * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt to be more secure than storing it in the Configuration.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param principal
-   *          a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param tokenFile
-   *          the path to the token file
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(JobConf job, String principal, String tokenFile) throws AccumuloSecurityException {
-    InputConfigurator.setConnectorInfo(CLASS, job, principal, tokenFile);
-  }
-
-  /**
-   * Determines if the connector has been configured.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return true if the connector has been configured, false otherwise
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static Boolean isConnectorInfoSet(JobConf job) {
-    return InputConfigurator.isConnectorInfoSet(CLASS, job);
-  }
-
-  /**
-   * Gets the user name from the configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the user name
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static String getPrincipal(JobConf job) {
-    return InputConfigurator.getPrincipal(CLASS, job);
-  }
-
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   * @see #setConnectorInfo(JobConf, String, String)
-   */
-  protected static AuthenticationToken getAuthenticationToken(JobConf job) {
-    return InputConfigurator.getAuthenticationToken(CLASS, job);
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param clientConfig
-   *          client configuration containing connection options
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(JobConf job, ClientConfiguration clientConfig) {
-    InputConfigurator.setZooKeeperInstance(CLASS, job, clientConfig);
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param instanceName
-   *          the Accumulo instance name
-   * @since 1.5.0
-   */
-  public static void setMockInstance(JobConf job, String instanceName) {
-    InputConfigurator.setMockInstance(CLASS, job, instanceName);
-  }
-
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return an Accumulo instance
-   * @since 1.5.0
-   * @see #setZooKeeperInstance(JobConf, ClientConfiguration)
-   * @see #setMockInstance(JobConf, String)
-   */
-  protected static Instance getInstance(JobConf job) {
-    return InputConfigurator.getInstance(CLASS, job);
-  }
-
-  /**
-   * Sets the log level for this job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param level
-   *          the logging level
-   * @since 1.5.0
-   */
-  public static void setLogLevel(JobConf job, Level level) {
-    InputConfigurator.setLogLevel(CLASS, job, level);
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the log level
-   * @since 1.5.0
-   * @see #setLogLevel(JobConf, Level)
-   */
-  protected static Level getLogLevel(JobConf job) {
-    return InputConfigurator.getLogLevel(CLASS, job);
-  }
-
-  /**
-   * Sets the {@link org.apache.accumulo.core.security.Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param auths
-   *          the user's authorizations
-   * @since 1.5.0
-   */
-  public static void setScanAuthorizations(JobConf job, Authorizations auths) {
-    InputConfigurator.setScanAuthorizations(CLASS, job, auths);
-  }
-
-  /**
-   * Gets the authorizations to set for the scans from the configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the Accumulo scan authorizations
-   * @since 1.5.0
-   * @see #setScanAuthorizations(JobConf, Authorizations)
-   */
-  protected static Authorizations getScanAuthorizations(JobConf job) {
-    return InputConfigurator.getScanAuthorizations(CLASS, job);
-  }
-
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return an Accumulo tablet locator
-   * @throws org.apache.accumulo.core.client.TableNotFoundException
-   *           if the table name set on the configuration doesn't exist
-   * @since 1.6.0
-   */
-  protected static TabletLocator getTabletLocator(JobConf job, String tableId) throws TableNotFoundException {
-    return InputConfigurator.getTabletLocator(CLASS, job, 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 InputFormat}.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @throws java.io.IOException
-   *           if the context is improperly configured
-   * @since 1.5.0
-   */
-  protected static void validateOptions(JobConf job) throws IOException {
-    InputConfigurator.validateOptions(CLASS, job);
-  }
-
-  /**
-   * Fetches all {@link InputTableConfig}s that have been set on the given Hadoop job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @return the {@link InputTableConfig} objects set on the job
-   * @since 1.6.0
-   */
-  public static Map<String,InputTableConfig> getInputTableConfigs(JobConf job) {
-    return InputConfigurator.getInputTableConfigs(CLASS, job);
-  }
-
-  /**
-   * Fetches a {@link InputTableConfig} that has been set on the configuration for a specific table.
-   * 
-   * <p>
-   * null is returned in the event that the table doesn't exist.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param tableName
-   *          the table name for which to grab the config object
-   * @return the {@link InputTableConfig} for the given table
-   * @since 1.6.0
-   */
-  public static InputTableConfig getInputTableConfig(JobConf job, String tableName) {
-    return InputConfigurator.getInputTableConfig(CLASS, job, tableName);
-  }
-
-  /**
-   * An abstract base class to be used to create {@link org.apache.hadoop.mapred.RecordReader} instances that convert from Accumulo
-   * {@link org.apache.accumulo.core.data.Key}/{@link org.apache.accumulo.core.data.Value} pairs to the user's K/V types.
-   * 
-   * Subclasses must implement {@link #next(Object, Object)} to update key and value, and also to update the following variables:
-   * <ul>
-   * <li>Key {@link #currentKey} (used for progress reporting)</li>
-   * <li>int {@link #numKeysRead} (used for progress reporting)</li>
-   * </ul>
-   */
-  protected abstract static class AbstractRecordReader<K,V> implements RecordReader<K,V> {
-    protected long numKeysRead;
-    protected Iterator<Map.Entry<Key,Value>> scannerIterator;
-    protected RangeInputSplit split;
-
-    /**
-     * Configures the iterators on a scanner for the given table name.
-     * 
-     * @param job
-     *          the Hadoop job configuration
-     * @param scanner
-     *          the scanner for which to configure the iterators
-     * @param tableName
-     *          the table name for which the scanner is configured
-     * @since 1.6.0
-     */
-    protected abstract void setupIterators(JobConf job, Scanner scanner, String tableName, RangeInputSplit split);
-
-    /**
-     * Initialize a scanner over the given input split using this task attempt configuration.
-     */
-    public void initialize(InputSplit inSplit, JobConf job) throws IOException {
-      Scanner scanner;
-      split = (RangeInputSplit) inSplit;
-      log.debug("Initializing input split: " + split.getRange());
-
-      Instance instance = split.getInstance();
-      if (null == instance) {
-        instance = getInstance(job);
-      }
-
-      String principal = split.getPrincipal();
-      if (null == principal) {
-        principal = getPrincipal(job);
-      }
-
-      AuthenticationToken token = split.getToken();
-      if (null == token) {
-        token = getAuthenticationToken(job);
-      }
-
-      Authorizations authorizations = split.getAuths();
-      if (null == authorizations) {
-        authorizations = getScanAuthorizations(job);
-      }
-
-      String table = split.getTableName();
-
-      // in case the table name changed, we can still use the previous name for terms of configuration,
-      // but the scanner will use the table id resolved at job setup time
-      InputTableConfig tableConfig = getInputTableConfig(job, split.getTableName());
-
-      Boolean isOffline = split.isOffline();
-      if (null == isOffline) {
-        isOffline = tableConfig.isOfflineScan();
-      }
-
-      Boolean isIsolated = split.isIsolatedScan();
-      if (null == isIsolated) {
-        isIsolated = tableConfig.shouldUseIsolatedScanners();
-      }
-
-      Boolean usesLocalIterators = split.usesLocalIterators();
-      if (null == usesLocalIterators) {
-        usesLocalIterators = tableConfig.shouldUseLocalIterators();
-      }
-
-      List<IteratorSetting> iterators = split.getIterators();
-      if (null == iterators) {
-        iterators = tableConfig.getIterators();
-      }
-
-      Collection<Pair<Text,Text>> columns = split.getFetchedColumns();
-      if (null == columns) {
-        columns = tableConfig.getFetchedColumns();
-      }
-
-      try {
-        log.debug("Creating connector with user: " + principal);
-        log.debug("Creating scanner for table: " + table);
-        log.debug("Authorizations are: " + authorizations);
-        if (isOffline) {
-          scanner = new OfflineScanner(instance, new Credentials(principal, token), split.getTableId(), authorizations);
-        } else if (instance instanceof MockInstance) {
-          scanner = instance.getConnector(principal, token).createScanner(split.getTableName(), authorizations);
-        } else {
-          scanner = new ScannerImpl(instance, new Credentials(principal, token), split.getTableId(), authorizations);
-        }
-        if (isIsolated) {
-          log.info("Creating isolated scanner");
-          scanner = new IsolatedScanner(scanner);
-        }
-        if (usesLocalIterators) {
-          log.info("Using local iterators");
-          scanner = new ClientSideIteratorScanner(scanner);
-        }
-        setupIterators(job, scanner, split.getTableName(), split);
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-
-      // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : columns) {
-        if (c.getSecond() != null) {
-          log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
-          scanner.fetchColumn(c.getFirst(), c.getSecond());
-        } else {
-          log.debug("Fetching column family " + c.getFirst());
-          scanner.fetchColumnFamily(c.getFirst());
-        }
-      }
-
-      scanner.setRange(split.getRange());
-
-      numKeysRead = 0;
-
-      // do this last after setting all scanner options
-      scannerIterator = scanner.iterator();
-    }
-
-    @Override
-    public void close() {}
-
-    @Override
-    public long getPos() throws IOException {
-      return numKeysRead;
-    }
-
-    @Override
-    public float getProgress() throws IOException {
-      if (numKeysRead > 0 && currentKey == null)
-        return 1.0f;
-      return split.getProgress(currentKey);
-    }
-
-    protected Key currentKey = null;
-
-  }
-
-  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf job, String tableId, List<Range> ranges) throws TableNotFoundException, AccumuloException,
-      AccumuloSecurityException {
-
-    Instance instance = getInstance(job);
-    Connector conn = instance.getConnector(getPrincipal(job), getAuthenticationToken(job));
-
-    return InputConfigurator.binOffline(tableId, ranges, instance, conn);
-  }
-
-  /**
-   * Read the metadata table to get tablets and match up ranges to them.
-   */
-  @Override
-  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
-    Level logLevel = getLogLevel(job);
-    log.setLevel(logLevel);
-    validateOptions(job);
-
-    Random random = new Random();
-    LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
-    Map<String,InputTableConfig> tableConfigs = getInputTableConfigs(job);
-    for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
-      String tableName = tableConfigEntry.getKey();
-      InputTableConfig tableConfig = tableConfigEntry.getValue();
-
-      Instance instance = getInstance(job);
-      boolean mockInstance;
-      String tableId;
-      // resolve table name to id once, and use id from this point forward
-      if (instance instanceof MockInstance) {
-        tableId = "";
-        mockInstance = true;
-      } else {
-        try {
-          tableId = Tables.getTableId(instance, tableName);
-        } catch (TableNotFoundException e) {
-          throw new IOException(e);
-        }
-        mockInstance = false;
-      }
-
-      Authorizations auths = getScanAuthorizations(job);
-      String principal = getPrincipal(job);
-      AuthenticationToken token = getAuthenticationToken(job);
-
-      boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
-      List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
-      if (ranges.isEmpty()) {
-        ranges = new ArrayList<Range>(1);
-        ranges.add(new Range());
-      }
-
-      // get the metadata information for these ranges
-      Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
-      TabletLocator tl;
-      try {
-        if (tableConfig.isOfflineScan()) {
-          binnedRanges = binOfflineTable(job, tableId, ranges);
-          while (binnedRanges == null) {
-            // Some tablets were still online, try again
-            UtilWaitThread.sleep(100 + random.nextInt(100)); // sleep randomly between 100 and 200 ms
-            binnedRanges = binOfflineTable(job, tableId, ranges);
-          }
-        } else {
-          tl = getTabletLocator(job, tableId);
-          // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
-          tl.invalidateCache();
-          Credentials creds = new Credentials(getPrincipal(job), getAuthenticationToken(job));
-
-          while (!tl.binRanges(creds, ranges, binnedRanges).isEmpty()) {
-            if (!(instance instanceof MockInstance)) {
-              if (!Tables.exists(instance, tableId))
-                throw new TableDeletedException(tableId);
-              if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
-                throw new TableOfflineException(instance, tableId);
-            }
-            binnedRanges.clear();
-            log.warn("Unable to locate bins for specified ranges. Retrying.");
-            UtilWaitThread.sleep(100 + random.nextInt(100)); // sleep randomly between 100 and 200 ms
-            tl.invalidateCache();
-          }
-        }
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-
-      HashMap<Range,ArrayList<String>> splitsToAdd = null;
-
-      if (!autoAdjust)
-        splitsToAdd = new HashMap<Range,ArrayList<String>>();
-
-      HashMap<String,String> hostNameCache = new HashMap<String,String>();
-      for (Map.Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
-        String ip = tserverBin.getKey().split(":", 2)[0];
-        String location = hostNameCache.get(ip);
-        if (location == null) {
-          InetAddress inetAddress = InetAddress.getByName(ip);
-          location = inetAddress.getCanonicalHostName();
-          hostNameCache.put(ip, location);
-        }
-        for (Map.Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
-          Range ke = extentRanges.getKey().toDataRange();
-          for (Range r : extentRanges.getValue()) {
-            if (autoAdjust) {
-              // divide ranges into smaller ranges, based on the tablets
-              RangeInputSplit split = new RangeInputSplit(tableName, tableId, ke.clip(r), new String[] {location});
-
-              split.setOffline(tableConfig.isOfflineScan());
-              split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-              split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-              split.setMockInstance(mockInstance);
-              split.setFetchedColumns(tableConfig.getFetchedColumns());
-              split.setPrincipal(principal);
-              split.setToken(token);
-              split.setInstanceName(instance.getInstanceName());
-              split.setZooKeepers(instance.getZooKeepers());
-              split.setAuths(auths);
-              split.setIterators(tableConfig.getIterators());
-              split.setLogLevel(logLevel);
-
-              splits.add(split);
-            } else {
-              // don't divide ranges
-              ArrayList<String> locations = splitsToAdd.get(r);
-              if (locations == null)
-                locations = new ArrayList<String>(1);
-              locations.add(location);
-              splitsToAdd.put(r, locations);
-            }
-          }
-        }
-      }
-
-      if (!autoAdjust)
-        for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
-          RangeInputSplit split = new RangeInputSplit(tableName, tableId, entry.getKey(), entry.getValue().toArray(new String[0]));
-
-          split.setOffline(tableConfig.isOfflineScan());
-          split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-          split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-          split.setMockInstance(mockInstance);
-          split.setFetchedColumns(tableConfig.getFetchedColumns());
-          split.setPrincipal(principal);
-          split.setToken(token);
-          split.setInstanceName(instance.getInstanceName());
-          split.setZooKeepers(instance.getZooKeepers());
-          split.setAuths(auths);
-          split.setIterators(tableConfig.getIterators());
-          split.setLogLevel(logLevel);
-
-          splits.add(split);
-        }
-    }
-
-    return splits.toArray(new InputSplit[splits.size()]);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
deleted file mode 100644
index 8a1d6df..0000000
--- a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.ArrayByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.commons.collections.map.LRUMap;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.util.Progressable;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to write output in the Accumulo data file format.<br />
- * Care should be taken to write only sorted data (sorted by {@link Key}), as this is an important requirement of Accumulo data files.
- * 
- * <p>
- * The output path to be created must be specified via {@link AccumuloFileOutputFormat#setOutputPath(JobConf, Path)}. This is inherited from
- * {@link FileOutputFormat#setOutputPath(JobConf, Path)}. Other methods from {@link FileOutputFormat} are not supported and may be ignored or cause failures.
- * Using other Hadoop configuration options that affect the behavior of the underlying files directly in the Job's configuration may work, but are not directly
- * supported at this time.
- */
-public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
-  
-  private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-  
-  /**
-   * This helper method provides an AccumuloConfiguration object constructed from the Accumulo defaults, and overridden with Accumulo properties that have been
-   * stored in the Job's configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @since 1.5.0
-   */
-  protected static AccumuloConfiguration getAccumuloConfiguration(JobConf job) {
-    return FileOutputConfigurator.getAccumuloConfiguration(CLASS, job);
-  }
-  
-  /**
-   * 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);
-  }
-  
-  @Override
-  public RecordWriter<Key,Value> getRecordWriter(FileSystem ignored, JobConf job, String name, Progressable progress) throws IOException {
-    // get the path of the temporary output file
-    final Configuration conf = job;
-    final AccumuloConfiguration acuConf = getAccumuloConfiguration(job);
-    
-    final String extension = acuConf.get(Property.TABLE_FILE_TYPE);
-    final Path file = new Path(getWorkOutputPath(job), getUniqueName(job, "part") + "." + extension);
-    
-    final LRUMap validVisibilities = new LRUMap(1000);
-    
-    return new RecordWriter<Key,Value>() {
-      FileSKVWriter out = null;
-      
-      @Override
-      public void close(Reporter reporter) throws IOException {
-        if (out != null)
-          out.close();
-      }
-      
-      @Override
-      public void write(Key key, Value value) throws IOException {
-        
-        Boolean wasChecked = (Boolean) validVisibilities.get(key.getColumnVisibilityData());
-        if (wasChecked == null) {
-          byte[] cv = key.getColumnVisibilityData().toArray();
-          new ColumnVisibility(cv);
-          validVisibilities.put(new ArrayByteSequence(Arrays.copyOf(cv, cv.length)), Boolean.TRUE);
-        }
-        
-        if (out == null) {
-          out = FileOperations.getInstance().openWriter(file.toString(), file.getFileSystem(conf), conf, acuConf);
-          out.startDefaultLocalityGroup();
-        }
-        out.append(key, value);
-      }
-    };
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
deleted file mode 100644
index 18e286a..0000000
--- a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.log4j.Level;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link InputFormat} provides keys and values of type {@link Key} and
- * {@link Value} to the Map function.
- * 
- * The user must specify the following via static configurator methods:
- * 
- * <ul>
- * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, ClientConfiguration)} OR
- * {@link AccumuloInputFormat#setMockInstance(JobConf, String)}
- * </ul>
- * 
- * Other static methods are optional.
- */
-public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
-
-  @Override
-  public RecordReader<Key,Value> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
-    log.setLevel(getLogLevel(job));
-    
-    // Override the log level from the configuration as if the RangeInputSplit has one it's the more correct one to use.
-    if (split instanceof org.apache.accumulo.core.client.mapreduce.RangeInputSplit) {
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit risplit = (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) split;
-      Level level = risplit.getLogLevel();
-      if (null != level) {
-        log.setLevel(level);
-      }
-    }
-    
-    RecordReaderBase<Key,Value> recordReader = new RecordReaderBase<Key,Value>() {
-
-      @Override
-      public boolean next(Key key, Value value) throws IOException {
-        if (scannerIterator.hasNext()) {
-          ++numKeysRead;
-          Entry<Key,Value> entry = scannerIterator.next();
-          key.set(currentKey = entry.getKey());
-          value.set(entry.getValue().get());
-          if (log.isTraceEnabled())
-            log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
-          return true;
-        }
-        return false;
-      }
-
-      @Override
-      public Key createKey() {
-        return new Key();
-      }
-
-      @Override
-      public Value createValue() {
-        return new Value();
-      }
-
-    };
-    recordReader.initialize(split, job);
-    return recordReader;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a8577a1c/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
deleted file mode 100644
index bbafef5..0000000
--- a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
-import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This class allows MapReduce jobs to use multiple Accumulo tables as the source of data. This {@link org.apache.hadoop.mapred.InputFormat} provides keys and
- * values of type {@link Key} and {@link Value} to the Map function.
- * 
- * The user must specify the following via static configurator methods:
- * 
- * <ul>
- * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, org.apache.accumulo.core.client.security.tokens.AuthenticationToken)}
- * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloInputFormat#setScanAuthorizations(JobConf, org.apache.accumulo.core.security.Authorizations)}
- * <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, ClientConfiguration)} OR {@link AccumuloInputFormat#setMockInstance(JobConf, String)}
- * <li>{@link AccumuloMultiTableInputFormat#setInputTableConfigs(org.apache.hadoop.mapred.JobConf, java.util.Map)}
- * </ul>
- * 
- * Other static methods are optional.
- */
-
-public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value> {
-
-  /**
-   * Sets the {@link InputTableConfig} objects on the given Hadoop configuration
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param configs
-   *          the table query configs to be set on the configuration.
-   * @since 1.6.0
-   */
-  public static void setInputTableConfigs(JobConf job, Map<String,InputTableConfig> configs) {
-    InputConfigurator.setInputTableConfigs(CLASS, job, configs);
-  }
-
-  @Override
-  public RecordReader<Key,Value> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
-    log.setLevel(getLogLevel(job));
-    InputFormatBase.RecordReaderBase<Key,Value> recordReader = new InputFormatBase.RecordReaderBase<Key,Value>() {
-
-      @Override
-      public boolean next(Key key, Value value) throws IOException {
-        if (scannerIterator.hasNext()) {
-          ++numKeysRead;
-          Map.Entry<Key,Value> entry = scannerIterator.next();
-          key.set(currentKey = entry.getKey());
-          value.set(entry.getValue().get());
-          if (log.isTraceEnabled())
-            log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
-          return true;
-        }
-        return false;
-      }
-
-      @Override
-      public Key createKey() {
-        return new Key();
-      }
-
-      @Override
-      public Value createValue() {
-        return new Value();
-      }
-
-    };
-    recordReader.initialize(split, job);
-    return recordReader;
-  }
-}