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

[01/19] Squashed commit of the following:

Updated Branches:
  refs/heads/1.5.1-SNAPSHOT 7eaedc45b -> 3a703d75d


http://git-wip-us.apache.org/repos/asf/accumulo/blob/45ae55fc/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
index ba647e9..7239b01 100644
--- a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
@@ -20,15 +20,17 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIterator;
 import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIteratorOption;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit;
 import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RegexType;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.data.Key;
@@ -36,6 +38,7 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -46,15 +49,16 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
 
 public class AccumuloInputFormatTest {
-  
+
   @After
   public void tearDown() throws Exception {}
-  
+
   /**
    * Test basic setting & getting of max versions.
    * 
@@ -68,7 +72,7 @@ public class AccumuloInputFormatTest {
     int version = AccumuloInputFormat.getMaxVersions(job.getConfiguration());
     assertEquals(1, version);
   }
-  
+
   /**
    * Test max versions with an invalid value.
    * 
@@ -80,7 +84,7 @@ public class AccumuloInputFormatTest {
     JobContext job = new JobContext(new Configuration(), new JobID());
     AccumuloInputFormat.setMaxVersions(job.getConfiguration(), 0);
   }
-  
+
   /**
    * Test no max version configured.
    */
@@ -89,7 +93,7 @@ public class AccumuloInputFormatTest {
     JobContext job = new JobContext(new Configuration(), new JobID());
     assertEquals(-1, AccumuloInputFormat.getMaxVersions(job.getConfiguration()));
   }
-  
+
   /**
    * Check that the iterator configuration is getting stored in the Job conf correctly.
    */
@@ -97,45 +101,45 @@ public class AccumuloInputFormatTest {
   @Test
   public void testSetIterator() {
     JobContext job = new JobContext(new Configuration(), new JobID());
-    
+
     AccumuloInputFormat.setIterator(job, 1, "org.apache.accumulo.core.iterators.WholeRowIterator", "WholeRow");
     Configuration conf = job.getConfiguration();
     String iterators = conf.get("AccumuloInputFormat.iterators");
     assertEquals("1:org.apache.accumulo.core.iterators.WholeRowIterator:WholeRow", iterators);
   }
-  
+
   @Test
   public void testAddIterator() {
     JobContext job = new JobContext(new Configuration(), new JobID());
-    
+
     AccumuloInputFormat.addIterator(job.getConfiguration(), new IteratorSetting(1, "WholeRow", WholeRowIterator.class));
     AccumuloInputFormat.addIterator(job.getConfiguration(), 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.getConfiguration(), iter);
-    
+
     List<AccumuloIterator> list = AccumuloInputFormat.getIterators(job.getConfiguration());
-    
+
     // Check the list size
     assertTrue(list.size() == 3);
-    
+
     // Walk the list and make sure our settings are correct
     AccumuloIterator setting = list.get(0);
     assertEquals(1, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.user.WholeRowIterator", setting.getIteratorClass());
     assertEquals("WholeRow", setting.getIteratorName());
-    
+
     setting = list.get(1);
     assertEquals(2, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
     assertEquals("Versions", setting.getIteratorName());
-    
+
     setting = list.get(2);
     assertEquals(3, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
     assertEquals("Count", setting.getIteratorName());
-    
+
     List<AccumuloIteratorOption> iteratorOptions = AccumuloInputFormat.getIteratorOptions(job.getConfiguration());
     assertEquals(2, iteratorOptions.size());
     assertEquals("Count", iteratorOptions.get(0).getIteratorName());
@@ -145,7 +149,7 @@ public class AccumuloInputFormatTest {
     assertEquals("junk", iteratorOptions.get(1).getKey());
     assertEquals("\0omg:!\\xyzzy", iteratorOptions.get(1).getValue());
   }
-  
+
   /**
    * 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.
@@ -160,16 +164,16 @@ public class AccumuloInputFormatTest {
     someSetting.addOption(key, value);
     Job job = new Job();
     AccumuloInputFormat.addIterator(job.getConfiguration(), someSetting);
-    
+
     final String rawConfigOpt = new AccumuloIteratorOption("iterator", key, value).toString();
-    
+
     assertEquals(rawConfigOpt, job.getConfiguration().get("AccumuloInputFormat.iterators.options"));
-    
+
     List<AccumuloIteratorOption> opts = AccumuloInputFormat.getIteratorOptions(job.getConfiguration());
     assertEquals(1, opts.size());
     assertEquals(opts.get(0).getKey(), key);
     assertEquals(opts.get(0).getValue(), value);
-    
+
     someSetting.addOption(key + "2", value);
     someSetting.setPriority(2);
     someSetting.setName("it2");
@@ -181,7 +185,7 @@ public class AccumuloInputFormatTest {
       assertEquals(opt.getValue(), value);
     }
   }
-  
+
   /**
    * Test getting iterator settings for multiple iterators set
    */
@@ -189,34 +193,34 @@ public class AccumuloInputFormatTest {
   @Test
   public void testGetIteratorSettings() {
     JobContext job = new JobContext(new Configuration(), new JobID());
-    
+
     AccumuloInputFormat.setIterator(job, 1, "org.apache.accumulo.core.iterators.WholeRowIterator", "WholeRow");
     AccumuloInputFormat.setIterator(job, 2, "org.apache.accumulo.core.iterators.VersioningIterator", "Versions");
     AccumuloInputFormat.setIterator(job, 3, "org.apache.accumulo.core.iterators.CountingIterator", "Count");
-    
+
     List<AccumuloIterator> list = AccumuloInputFormat.getIterators(job);
-    
+
     // Check the list size
     assertTrue(list.size() == 3);
-    
+
     // Walk the list and make sure our settings are correct
     AccumuloIterator setting = list.get(0);
     assertEquals(1, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.WholeRowIterator", setting.getIteratorClass());
     assertEquals("WholeRow", setting.getIteratorName());
-    
+
     setting = list.get(1);
     assertEquals(2, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
     assertEquals("Versions", setting.getIteratorName());
-    
+
     setting = list.get(2);
     assertEquals(3, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
     assertEquals("Count", setting.getIteratorName());
-    
+
   }
-  
+
   /**
    * Check that the iterator options are getting stored in the Job conf correctly.
    */
@@ -225,12 +229,12 @@ public class AccumuloInputFormatTest {
   public void testSetIteratorOption() {
     JobContext job = new JobContext(new Configuration(), new JobID());
     AccumuloInputFormat.setIteratorOption(job, "someIterator", "aKey", "aValue");
-    
+
     Configuration conf = job.getConfiguration();
     String options = conf.get("AccumuloInputFormat.iterators.options");
     assertEquals(new String("someIterator:aKey:aValue"), options);
   }
-  
+
   /**
    * Test getting iterator options for multiple options set
    */
@@ -238,49 +242,49 @@ public class AccumuloInputFormatTest {
   @Test
   public void testGetIteratorOption() {
     JobContext job = new JobContext(new Configuration(), new JobID());
-    
+
     AccumuloInputFormat.setIteratorOption(job, "iterator1", "key1", "value1");
     AccumuloInputFormat.setIteratorOption(job, "iterator2", "key2", "value2");
     AccumuloInputFormat.setIteratorOption(job, "iterator3", "key3", "value3");
-    
+
     List<AccumuloIteratorOption> list = AccumuloInputFormat.getIteratorOptions(job);
-    
+
     // Check the list size
     assertEquals(3, list.size());
-    
+
     // Walk the list and make sure all the options are correct
     AccumuloIteratorOption option = list.get(0);
     assertEquals("iterator1", option.getIteratorName());
     assertEquals("key1", option.getKey());
     assertEquals("value1", option.getValue());
-    
+
     option = list.get(1);
     assertEquals("iterator2", option.getIteratorName());
     assertEquals("key2", option.getKey());
     assertEquals("value2", option.getValue());
-    
+
     option = list.get(2);
     assertEquals("iterator3", option.getIteratorName());
     assertEquals("key3", option.getKey());
     assertEquals("value3", option.getValue());
   }
-  
+
   @SuppressWarnings("deprecation")
   @Test
   public void testSetRegex() {
     JobContext job = new JobContext(new Configuration(), new JobID());
-    
+
     String regex = ">\"*%<>\'\\";
-    
+
     AccumuloInputFormat.setRegex(job, RegexType.ROW, regex);
-    
+
     assertTrue(regex.equals(AccumuloInputFormat.getRegex(job, RegexType.ROW)));
   }
-  
+
   static class TestMapper extends Mapper<Key,Value,Key,Value> {
     Key key = null;
     int count = 0;
-    
+
     @Override
     protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
       if (key != null)
@@ -291,7 +295,7 @@ public class AccumuloInputFormatTest {
       count++;
     }
   }
-  
+
   @Test
   public void testMap() throws Exception {
     MockInstance mockInstance = new MockInstance("testmapinstance");
@@ -304,20 +308,27 @@ public class AccumuloInputFormatTest {
       bw.addMutation(m);
     }
     bw.close();
-    
+
     Job job = new Job(new Configuration());
     job.setInputFormatClass(AccumuloInputFormat.class);
     job.setMapperClass(TestMapper.class);
     job.setNumReduceTasks(0);
     AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
     AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
-    
+
     AccumuloInputFormat input = new AccumuloInputFormat();
     List<InputSplit> splits = input.getSplits(job);
     assertEquals(splits.size(), 1);
-    
+
     TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
     for (InputSplit split : splits) {
+      RangeInputSplit risplit = (RangeInputSplit) split;
+      Assert.assertEquals("root", risplit.getUsername());
+      Assert.assertArrayEquals(new byte[0], risplit.getPassword());
+      Assert.assertEquals("testtable", risplit.getTable());
+      Assert.assertEquals(new Authorizations(), risplit.getAuths());
+      Assert.assertEquals("testmapinstance", risplit.getInstanceName());
+      
       TaskAttemptID id = new TaskAttemptID();
       TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
       RecordReader<Key,Value> reader = input.createRecordReader(split, attempt);
@@ -326,7 +337,7 @@ public class AccumuloInputFormatTest {
       mapper.run(context);
     }
   }
-  
+
   @Test
   public void testSimple() throws Exception {
     MockInstance mockInstance = new MockInstance("testmapinstance");
@@ -339,7 +350,7 @@ public class AccumuloInputFormatTest {
       bw.addMutation(m);
     }
     bw.close();
-    
+
     JobContext job = new JobContext(new Configuration(), new JobID());
     AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable2", new Authorizations());
     AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
@@ -348,14 +359,14 @@ public class AccumuloInputFormatTest {
     TaskAttemptContext tac = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID());
     RecordReader<Key,Value> rr = input.createRecordReader(ris, tac);
     rr.initialize(ris, tac);
-    
+
     TestMapper mapper = new TestMapper();
     Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), tac.getTaskAttemptID(), rr, null, null, null, ris);
     while (rr.nextKeyValue()) {
       mapper.map(rr.getCurrentKey(), rr.getCurrentValue(), context);
     }
   }
-  
+
   @SuppressWarnings("deprecation")
   @Test
   public void testRegex() throws Exception {
@@ -369,7 +380,7 @@ public class AccumuloInputFormatTest {
       bw.addMutation(m);
     }
     bw.close();
-    
+
     JobContext job = new JobContext(new Configuration(), new JobID());
     AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable3", new Authorizations());
     AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
@@ -380,10 +391,144 @@ public class AccumuloInputFormatTest {
     TaskAttemptContext tac = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID());
     RecordReader<Key,Value> rr = input.createRecordReader(ris, tac);
     rr.initialize(ris, tac);
-    
+
     Pattern p = Pattern.compile(regex);
     while (rr.nextKeyValue()) {
-      Assert.assertTrue( p.matcher( rr.getCurrentKey().getRow().toString()).matches());
+      Assert.assertTrue(p.matcher(rr.getCurrentKey().getRow().toString()).matches());
+    }
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testCorrectRangeInputSplits() throws Exception {
+    JobContext job = new JobContext(new Configuration(), new JobID());
+
+    String username = "user", table = "table", rowRegex = "row.*", colfRegex = "colf.*", colqRegex = "colq.*";
+    String valRegex = "val.*", instance = "instance";
+    byte[] password = "password".getBytes();
+    Authorizations auths = new Authorizations("foo");
+    Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
+    boolean isolated = true, localIters = true;
+    int maxVersions = 5;
+    Level level = Level.WARN;
+
+    Instance inst = new MockInstance(instance);
+    Connector connector = inst.getConnector(username, password);
+    connector.tableOperations().create(table);
+
+    AccumuloInputFormat.setInputInfo(job, username, password, table, auths);
+    AccumuloInputFormat.setMockInstance(job, instance);
+    AccumuloInputFormat.setRegex(job, RegexType.ROW, rowRegex);
+    AccumuloInputFormat.setRegex(job, RegexType.COLUMN_FAMILY, colfRegex);
+    AccumuloInputFormat.setRegex(job, RegexType.COLUMN_QUALIFIER, colqRegex);
+    AccumuloInputFormat.setRegex(job, RegexType.VALUE, valRegex);
+    AccumuloInputFormat.setIsolated(job, isolated);
+    AccumuloInputFormat.setLocalIterators(job, localIters);
+    AccumuloInputFormat.setMaxVersions(job, maxVersions);
+    AccumuloInputFormat.fetchColumns(job, fetchColumns);
+    AccumuloInputFormat.setLogLevel(job, level);
+    
+    AccumuloInputFormat aif = new AccumuloInputFormat();
+    
+    List<InputSplit> splits = aif.getSplits(job);
+    
+    Assert.assertEquals(1, splits.size());
+    
+    InputSplit split = splits.get(0);
+    
+    Assert.assertEquals(RangeInputSplit.class, split.getClass());
+    
+    RangeInputSplit risplit = (RangeInputSplit) split;
+    
+    Assert.assertEquals(username, risplit.getUsername());
+    Assert.assertEquals(table, risplit.getTable());
+    Assert.assertArrayEquals(password, risplit.getPassword());
+    Assert.assertEquals(auths, risplit.getAuths());
+    Assert.assertEquals(instance, risplit.getInstanceName());
+    Assert.assertEquals(rowRegex, risplit.getRowRegex());
+    Assert.assertEquals(colfRegex, risplit.getColfamRegex());
+    Assert.assertEquals(colqRegex, risplit.getColqualRegex());
+    Assert.assertEquals(valRegex, risplit.getValueRegex());
+    Assert.assertEquals(isolated, risplit.isIsolatedScan());
+    Assert.assertEquals(localIters, risplit.usesLocalIterators());
+    Assert.assertEquals(maxVersions, risplit.getMaxVersions().intValue());
+    Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());
+    Assert.assertEquals(level, risplit.getLogLevel());
+  }
+
+  @Test
+  public void testPartialInputSplitDelegationToConfiguration() throws Exception {
+    MockInstance mockInstance = new MockInstance("testPartialInputSplitDelegationToConfiguration");
+    Connector c = mockInstance.getConnector("root", new byte[] {});
+    c.tableOperations().create("testtable");
+    BatchWriter bw = c.createBatchWriter("testtable", 10000L, 1000L, 4);
+    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();
+
+    Job job = new Job(new Configuration());
+    job.setInputFormatClass(AccumuloInputFormat.class);
+    job.setMapperClass(TestMapper.class);
+    job.setNumReduceTasks(0);
+    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
+    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testPartialInputSplitDelegationToConfiguration");
+
+    AccumuloInputFormat input = new AccumuloInputFormat();
+    List<InputSplit> splits = input.getSplits(job);
+    assertEquals(splits.size(), 1);
+
+    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
+    
+    RangeInputSplit emptySplit = new RangeInputSplit();
+    
+    // Using an empty split should fall back to the information in the Job's Configuration
+    TaskAttemptID id = new TaskAttemptID();
+    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
+    RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
+    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
+    reader.initialize(emptySplit, context);
+    mapper.run(context);
+  }
+
+  @Test(expected = IOException.class)
+  public void testPartialFailedInputSplitDelegationToConfiguration() throws Exception {
+    MockInstance mockInstance = new MockInstance("testPartialFailedInputSplitDelegationToConfiguration");
+    Connector c = mockInstance.getConnector("root", new byte[] {});
+    c.tableOperations().create("testtable");
+    BatchWriter bw = c.createBatchWriter("testtable", 10000L, 1000L, 4);
+    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();
+
+    Job job = new Job(new Configuration());
+    job.setInputFormatClass(AccumuloInputFormat.class);
+    job.setMapperClass(TestMapper.class);
+    job.setNumReduceTasks(0);
+    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
+    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testPartialFailedInputSplitDelegationToConfiguration");
+
+    AccumuloInputFormat input = new AccumuloInputFormat();
+    List<InputSplit> splits = input.getSplits(job);
+    assertEquals(splits.size(), 1);
+
+    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
+    
+    RangeInputSplit emptySplit = new RangeInputSplit();
+    emptySplit.setUsername("root");
+    emptySplit.setPassword("anythingelse".getBytes());
+    
+    // Using an empty split should fall back to the information in the Job's Configuration
+    TaskAttemptID id = new TaskAttemptID();
+    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
+    RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
+    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
+    reader.initialize(emptySplit, context);
+    mapper.run(context);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/45ae55fc/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
index 0673f1b..d9f9da0 100644
--- a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
@@ -27,7 +27,6 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/45ae55fc/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
new file mode 100644
index 0000000..22fb6e1
--- /dev/null
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
@@ -0,0 +1,100 @@
+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.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(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.getRange(), newSplit.getRange());
+    Assert.assertTrue(Arrays.equals(split.getLocations(), newSplit.getLocations()));
+  }
+
+
+
+  @Test
+  public void testAllFieldsWritable() throws IOException {
+    RangeInputSplit split = new RangeInputSplit(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.setMaxVersions(5);
+    split.setRowRegex("row");
+    split.setColfamRegex("colf");
+    split.setColqualRegex("colq");
+    split.setValueRegex("value");
+    split.setFetchedColumns(fetchedColumns);
+    split.setPassword("password".getBytes());
+    split.setUsername("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.getMaxVersions(), newSplit.getMaxVersions());
+    Assert.assertEquals(split.getRowRegex(), newSplit.getRowRegex());
+    Assert.assertEquals(split.getColfamRegex(), newSplit.getColfamRegex());
+    Assert.assertEquals(split.getColqualRegex(), newSplit.getColqualRegex());
+    Assert.assertEquals(split.getValueRegex(), newSplit.getValueRegex());
+    Assert.assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
+    Assert.assertEquals(new String(split.getPassword()), new String(newSplit.getPassword()));
+    Assert.assertEquals(split.getUsername(), newSplit.getUsername());
+    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/45ae55fc/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
----------------------------------------------------------------------
diff --git a/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java b/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
index c31c738..af12302 100644
--- a/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
+++ b/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
@@ -30,15 +30,13 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit;
+import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
 import org.apache.accumulo.core.client.mock.MockInstance;
 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.ColumnVisibility;
-import org.apache.accumulo.examples.simple.filedata.ChunkInputFormat;
-import org.apache.accumulo.examples.simple.filedata.ChunkInputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobID;


[08/19] git commit: Merge branch 'ACCUMULO-1854-merge' into ACCUMULO-1854-1.5-merge

Posted by el...@apache.org.
Merge branch 'ACCUMULO-1854-merge' into ACCUMULO-1854-1.5-merge

Conflicts:
	src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/4dfe3e01
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/4dfe3e01
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/4dfe3e01

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 4dfe3e01719696c483d65d62a42be11cb067746a
Parents: 16a2e0f 3beb9f7
Author: Josh Elser <el...@apache.org>
Authored: Thu Nov 21 20:20:02 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Nov 21 20:20:02 2013 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------



[18/19] git commit: ACCUMULO-1854 Remove todo and finish toString

Posted by el...@apache.org.
ACCUMULO-1854 Remove todo and finish toString


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/e1dd6f9b
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/e1dd6f9b
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/e1dd6f9b

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: e1dd6f9b28fa579eada4334b7072fca4155add15
Parents: 8dd3ae4
Author: Josh Elser <el...@apache.org>
Authored: Fri Nov 22 15:14:23 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Nov 22 15:17:58 2013 -0500

----------------------------------------------------------------------
 .../core/client/mapreduce/RangeInputSplit.java   | 19 ++++++++++++++++++-
 1 file changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e1dd6f9b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index e372801..69f2b38 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -311,7 +311,24 @@ public class RangeInputSplit extends InputSplit implements Writable {
     sb.append("Range: ").append(range);
     sb.append(" Locations: ").append(locations);
     sb.append(" Table: ").append(table);
-    // TODO finish building of string
+    sb.append(" InstanceName: ").append(instanceName);
+    sb.append(" zooKeepers: ").append(zooKeepers);
+    sb.append(" username: ").append(username);
+    sb.append(" password: ").append(new String(password));
+    sb.append(" Authorizations: ").append(auths);
+    sb.append(" offlineScan: ").append(offline);
+    sb.append(" mockInstance: ").append(mockInstance);
+    sb.append(" isolatedScan: ").append(isolatedScan);
+    sb.append(" localIterators: ").append(localIterators);
+    sb.append(" maxVersions: ").append(maxVersions);
+    sb.append(" rowRegex: ").append(rowRegex);
+    sb.append(" colfamRegex: ").append(colfamRegex);
+    sb.append(" colqualRegex: ").append(colqualRegex);
+    sb.append(" valueRegex: ").append(valueRegex);
+    sb.append(" fetchColumns: ").append(fetchedColumns);
+    sb.append(" iterators: ").append(iterators);
+    sb.append(" options: ").append(options);
+    sb.append(" logLevel: ").append(level);
     return sb.toString();
   }
 


[04/19] WIP Merge branch 'ACCUMULO-1854-merge' into ACCUMULO-1854-1.5-merge

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/16a2e0f0/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
index c8731b1,0000000..ff14107
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
@@@ -1,532 -1,0 +1,549 @@@
 +/*
 + * 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.util;
 +
 +import java.io.ByteArrayInputStream;
 +import java.io.ByteArrayOutputStream;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.nio.charset.Charset;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Set;
 +import java.util.StringTokenizer;
 +
 +import org.apache.accumulo.core.Constants;
 +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.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.mock.MockTabletLocator;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.core.util.ArgumentChecker;
 +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.Text;
 +import org.apache.hadoop.util.StringUtils;
 +
 +/**
 + * @since 1.5.0
 + */
 +public class InputConfigurator extends ConfiguratorBase {
 +  
 +  /**
 +   * Configuration keys for {@link Scanner}.
 +   * 
 +   * @since 1.5.0
 +   */
 +  public static enum ScanOpts {
 +    TABLE_NAME, AUTHORIZATIONS, RANGES, COLUMNS, ITERATORS
 +  }
 +  
 +  /**
 +   * Configuration keys for various features.
 +   * 
 +   * @since 1.5.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.5.0
 +   */
 +  public static void setInputTableName(Class<?> implementingClass, Configuration conf, String tableName) {
 +    ArgumentChecker.notNull(tableName);
 +    conf.set(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME), tableName);
 +  }
 +  
 +  /**
 +   * Gets the 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 table name
 +   * @since 1.5.0
 +   * @see #setInputTableName(Class, Configuration, String)
 +   */
 +  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.5.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.5.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 ? Constants.NO_AUTHS : new Authorizations(authString.getBytes());
 +  }
 +  
 +  /**
 +   * Sets the input ranges to scan 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
 +   * @since 1.5.0
 +   */
 +  public static void setRanges(Class<?> implementingClass, Configuration conf, Collection<Range> ranges) {
 +    ArgumentChecker.notNull(ranges);
 +    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()), Charset.forName("UTF-8")));
 +      }
 +    } catch (IOException ex) {
 +      throw new IllegalArgumentException("Unable to encode ranges to Base64", ex);
 +    }
 +    conf.setStrings(enumToConfKey(implementingClass, ScanOpts.RANGES), rangeStrings.toArray(new String[0]));
 +  }
 +  
 +  /**
 +   * 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.5.0
 +   * @see #setRanges(Class, Configuration, Collection)
 +   */
 +  public static List<Range> getRanges(Class<?> implementingClass, Configuration conf) throws IOException {
 +    ArrayList<Range> ranges = new ArrayList<Range>();
 +    for (String rangeString : conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.RANGES))) {
 +      ByteArrayInputStream bais = new ByteArrayInputStream(Base64.decodeBase64(rangeString.getBytes(Charset.forName("UTF-8"))));
 +      Range range = new Range();
 +      range.readFields(new DataInputStream(bais));
 +      ranges.add(range);
 +    }
 +    return ranges;
 +  }
 +  
 +  /**
 +   * Restricts the columns that will be mapped over for 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.
 +   * @since 1.5.0
 +   */
 +  public static void fetchColumns(Class<?> implementingClass, Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 +    ArgumentChecker.notNull(columnFamilyColumnQualifierPairs);
++    String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
++    conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings);
++  }
++
++  public static String[] serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
++    ArgumentChecker.notNull(columnFamilyColumnQualifierPairs);
 +    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())), Charset.forName("UTF-8"));
++
++      String col = new String(Base64.encodeBase64(TextUtil.getBytes(column.getFirst())));
 +      if (column.getSecond() != null)
-         col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())), Charset.forName("UTF-8"));
++        col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())));
 +      columnStrings.add(col);
 +    }
-     conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings.toArray(new String[0]));
++
++    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.5.0
 +   * @see #fetchColumns(Class, Configuration, Collection)
 +   */
 +  public static Set<Pair<Text,Text>> getFetchedColumns(Class<?> implementingClass, Configuration conf) {
++    ArgumentChecker.notNull(conf);
++
++    return deserializeFetchedColumns(conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.COLUMNS)));
++  }
++
++  public static Set<Pair<Text,Text>> deserializeFetchedColumns(Collection<String> serialized) {
 +    Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
-     for (String col : conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.COLUMNS))) {
++
++    if (null == serialized) {
++      return columns;
++    }
++
++    for (String col : serialized) {
 +      int idx = col.indexOf(":");
-       Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes(Charset.forName("UTF-8"))) : Base64.decodeBase64(col.substring(0, idx).getBytes(
-           Charset.forName("UTF-8"))));
++      Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes(Charset.forName("UTF-8"))) : Base64.decodeBase64(col.substring(0, idx).getBytes(Charset.forName("UTF-8"))));
 +      Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes()));
 +      columns.add(new Pair<Text,Text>(cf, cq));
 +    }
 +    return columns;
 +  }
 +  
 +  /**
 +   * Encode an iterator on the input for 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
 +   * @since 1.5.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()), Charset.forName("UTF-8"));
 +      baos.close();
 +    } catch (IOException e) {
 +      throw new IllegalArgumentException("unable to serialize IteratorSetting");
 +    }
 +    
 +    String iterators = conf.get(enumToConfKey(implementingClass, ScanOpts.ITERATORS));
 +    // 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(enumToConfKey(implementingClass, ScanOpts.ITERATORS), iterators);
 +  }
 +  
 +  /**
 +   * 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.5.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;
 +  }
 +  
 +  /**
 +   * 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.5.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.5.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.5.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.5.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.5.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.5.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. The accumulo-site.xml may need to be on the mapper's classpath if HDFS or the Accumulo directory in HDFS are non-standard.
 +   * 
 +   * <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.5.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.5.0
 +   * @see #setOfflineTableScan(Class, Configuration, boolean)
 +   */
 +  public static Boolean isOfflineScan(Class<?> implementingClass, Configuration conf) {
 +    return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), false);
 +  }
 +  
 +  /**
 +   * 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
 +   * @return an Accumulo tablet locator
 +   * @throws TableNotFoundException
 +   *           if the table name set on the configuration doesn't exist
 +   * @since 1.5.0
 +   */
 +  public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf) throws TableNotFoundException {
 +    String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
 +    if ("MockInstance".equals(instanceType))
 +      return new MockTabletLocator();
 +    Instance instance = getInstance(implementingClass, conf);
 +    String tableName = getInputTableName(implementingClass, conf);
 +    return TabletLocator.getInstance(instance, new Text(Tables.getTableId(instance, tableName)));
 +  }
 +  
 +  // 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.5.0
 +   */
 +  public static void validateOptions(Class<?> implementingClass, Configuration conf) throws IOException {
 +    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 {
 +      Connector c = getInstance(implementingClass, conf).getConnector(getPrincipal(implementingClass, conf),
 +          CredentialHelper.extractToken(getTokenClass(implementingClass, conf), getToken(implementingClass, conf)));
 +      if (!c.securityOperations().authenticateUser(getPrincipal(implementingClass, conf),
 +          CredentialHelper.extractToken(getTokenClass(implementingClass, conf), getToken(implementingClass, conf))))
 +        throw new IOException("Unable to authenticate user");
 +      if (!c.securityOperations().hasTablePermission(getPrincipal(implementingClass, conf), getInputTableName(implementingClass, conf), TablePermission.READ))
 +        throw new IOException("Unable to access table");
 +      
 +      if (!conf.getBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), false)) {
 +        // validate that any scan-time iterators can be loaded by the the tablet servers
 +        for (IteratorSetting iter : getIterators(implementingClass, conf)) {
 +          if (!c.tableOperations().testClassLoad(getInputTableName(implementingClass, conf), 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);
 +    }
 +  }
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/16a2e0f0/core/src/main/java/org/apache/accumulo/core/security/CredentialHelper.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/security/CredentialHelper.java
index 69e3ba1,0000000..5ebab3f
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/security/CredentialHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/CredentialHelper.java
@@@ -1,131 -1,0 +1,131 @@@
 +/*
 + * 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.security;
 +
 +import java.io.ByteArrayInputStream;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.nio.charset.Charset;
 +
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.commons.codec.binary.Base64;
 +import org.apache.commons.io.output.ByteArrayOutputStream;
 +import org.apache.log4j.Logger;
 +import org.apache.thrift.TDeserializer;
 +import org.apache.thrift.TException;
 +import org.apache.thrift.TSerializer;
 +
 +public class CredentialHelper {
 +  static Logger log = Logger.getLogger(CredentialHelper.class);
 +  
 +  public static TCredentials create(String principal, AuthenticationToken token, String instanceID) throws AccumuloSecurityException {
 +    String className = token.getClass().getName();
 +    return new TCredentials(principal, className, ByteBuffer.wrap(toBytes(token)), instanceID);
 +  }
 +  
 +  public static String asBase64String(TCredentials cred) throws AccumuloSecurityException {
 +    return new String(Base64.encodeBase64(asByteArray(cred)), Charset.forName("UTF-8"));
 +  }
 +  
 +  public static byte[] asByteArray(TCredentials cred) throws AccumuloSecurityException {
 +    TSerializer ts = new TSerializer();
 +    try {
 +      return ts.serialize(cred);
 +    } catch (TException e) {
 +      // This really shouldn't happen
 +      log.error(e, e);
 +      throw new AccumuloSecurityException(cred.getPrincipal(), SecurityErrorCode.SERIALIZATION_ERROR);
 +    }
 +  }
 +  
 +  public static TCredentials fromBase64String(String string) throws AccumuloSecurityException {
 +    return fromByteArray(Base64.decodeBase64(string.getBytes(Charset.forName("UTF-8"))));
 +  }
 +  
 +  public static TCredentials fromByteArray(byte[] serializedCredential) throws AccumuloSecurityException {
 +    if (serializedCredential == null)
 +      return null;
 +    TDeserializer td = new TDeserializer();
 +    try {
 +      TCredentials toRet = new TCredentials();
 +      td.deserialize(toRet, serializedCredential);
 +      return toRet;
 +    } catch (TException e) {
 +      // This really shouldn't happen
 +      log.error(e, e);
 +      throw new AccumuloSecurityException("unknown", SecurityErrorCode.SERIALIZATION_ERROR);
 +    }
 +  }
 +  
 +  public static AuthenticationToken extractToken(TCredentials toAuth) throws AccumuloSecurityException {
 +    return extractToken(toAuth.tokenClassName, toAuth.getToken());
 +  }
 +  
 +  public static TCredentials createSquelchError(String principal, AuthenticationToken token, String instanceID) {
 +    try {
 +      return create(principal, token, instanceID);
 +    } catch (AccumuloSecurityException e) {
 +      log.error(e, e);
 +      return null;
 +    }
 +  }
 +  
 +  public static String tokenAsBase64(AuthenticationToken token) throws AccumuloSecurityException {
 +    return new String(Base64.encodeBase64(toBytes(token)), Charset.forName("UTF-8"));
 +  }
 +  
-   private static byte[] toBytes(AuthenticationToken token) throws AccumuloSecurityException {
++  public static byte[] toBytes(AuthenticationToken token) throws AccumuloSecurityException {
 +    try {
 +      ByteArrayOutputStream bais = new ByteArrayOutputStream();
 +      token.write(new DataOutputStream(bais));
 +      byte[] serializedToken = bais.toByteArray();
 +      bais.close();
 +      return serializedToken;
 +    } catch (IOException e) {
 +      log.error(e, e);
 +      throw new AccumuloSecurityException("unknown", SecurityErrorCode.SERIALIZATION_ERROR);
 +    }
 +    
 +  }
 +  
 +  public static AuthenticationToken extractToken(String tokenClass, byte[] token) throws AccumuloSecurityException {
 +    try {
 +      Object obj = Class.forName(tokenClass).newInstance();
 +      if (obj instanceof AuthenticationToken) {
 +        AuthenticationToken toRet = (AuthenticationToken) obj;
 +        toRet.readFields(new DataInputStream(new ByteArrayInputStream(token)));
 +        return toRet;
 +      }
 +    } catch (ClassNotFoundException cnfe) {
 +      log.error(cnfe, cnfe);
 +    } catch (InstantiationException e) {
 +      log.error(e, e);
 +    } catch (IllegalAccessException e) {
 +      log.error(e, e);
 +    } catch (IOException e) {
 +      log.error(e, e);
 +      throw new AccumuloSecurityException("unknown", SecurityErrorCode.SERIALIZATION_ERROR);
 +    }
 +    throw new AccumuloSecurityException("unknown", SecurityErrorCode.INVALID_TOKEN);
 +  }
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/16a2e0f0/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest1.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest1.java
index 0000000,0000000..7239b01
new file mode 100644
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest1.java
@@@ -1,0 -1,0 +1,534 @@@
++/*
++ * 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.assertTrue;
++
++import java.io.IOException;
++import java.util.Collection;
++import java.util.Collections;
++import java.util.List;
++import java.util.regex.Pattern;
++
++import org.apache.accumulo.core.client.BatchWriter;
++import org.apache.accumulo.core.client.Connector;
++import org.apache.accumulo.core.client.Instance;
++import org.apache.accumulo.core.client.IteratorSetting;
++import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIterator;
++import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIteratorOption;
++import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RegexType;
++import org.apache.accumulo.core.client.mock.MockInstance;
++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.WholeRowIterator;
++import org.apache.accumulo.core.security.Authorizations;
++import org.apache.accumulo.core.util.Pair;
++import org.apache.hadoop.conf.Configuration;
++import org.apache.hadoop.io.Text;
++import org.apache.hadoop.mapreduce.InputSplit;
++import org.apache.hadoop.mapreduce.Job;
++import org.apache.hadoop.mapreduce.JobContext;
++import org.apache.hadoop.mapreduce.JobID;
++import org.apache.hadoop.mapreduce.Mapper;
++import org.apache.hadoop.mapreduce.RecordReader;
++import org.apache.hadoop.mapreduce.TaskAttemptContext;
++import org.apache.hadoop.mapreduce.TaskAttemptID;
++import org.apache.log4j.Level;
++import org.junit.After;
++import org.junit.Assert;
++import org.junit.Test;
++
++public class AccumuloInputFormatTest {
++
++  @After
++  public void tearDown() throws Exception {}
++
++  /**
++   * Test basic setting & getting of max versions.
++   * 
++   * @throws IOException
++   *           Signals that an I/O exception has occurred.
++   */
++  @Test
++  public void testMaxVersions() throws IOException {
++    JobContext job = new JobContext(new Configuration(), new JobID());
++    AccumuloInputFormat.setMaxVersions(job.getConfiguration(), 1);
++    int version = AccumuloInputFormat.getMaxVersions(job.getConfiguration());
++    assertEquals(1, version);
++  }
++
++  /**
++   * Test max versions with an invalid value.
++   * 
++   * @throws IOException
++   *           Signals that an I/O exception has occurred.
++   */
++  @Test(expected = IOException.class)
++  public void testMaxVersionsLessThan1() throws IOException {
++    JobContext job = new JobContext(new Configuration(), new JobID());
++    AccumuloInputFormat.setMaxVersions(job.getConfiguration(), 0);
++  }
++
++  /**
++   * Test no max version configured.
++   */
++  @Test
++  public void testNoMaxVersion() {
++    JobContext job = new JobContext(new Configuration(), new JobID());
++    assertEquals(-1, AccumuloInputFormat.getMaxVersions(job.getConfiguration()));
++  }
++
++  /**
++   * Check that the iterator configuration is getting stored in the Job conf correctly.
++   */
++  @SuppressWarnings("deprecation")
++  @Test
++  public void testSetIterator() {
++    JobContext job = new JobContext(new Configuration(), new JobID());
++
++    AccumuloInputFormat.setIterator(job, 1, "org.apache.accumulo.core.iterators.WholeRowIterator", "WholeRow");
++    Configuration conf = job.getConfiguration();
++    String iterators = conf.get("AccumuloInputFormat.iterators");
++    assertEquals("1:org.apache.accumulo.core.iterators.WholeRowIterator:WholeRow", iterators);
++  }
++
++  @Test
++  public void testAddIterator() {
++    JobContext job = new JobContext(new Configuration(), new JobID());
++
++    AccumuloInputFormat.addIterator(job.getConfiguration(), new IteratorSetting(1, "WholeRow", WholeRowIterator.class));
++    AccumuloInputFormat.addIterator(job.getConfiguration(), 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.getConfiguration(), iter);
++
++    List<AccumuloIterator> list = AccumuloInputFormat.getIterators(job.getConfiguration());
++
++    // Check the list size
++    assertTrue(list.size() == 3);
++
++    // Walk the list and make sure our settings are correct
++    AccumuloIterator setting = list.get(0);
++    assertEquals(1, setting.getPriority());
++    assertEquals("org.apache.accumulo.core.iterators.user.WholeRowIterator", setting.getIteratorClass());
++    assertEquals("WholeRow", setting.getIteratorName());
++
++    setting = list.get(1);
++    assertEquals(2, setting.getPriority());
++    assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
++    assertEquals("Versions", setting.getIteratorName());
++
++    setting = list.get(2);
++    assertEquals(3, setting.getPriority());
++    assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
++    assertEquals("Count", setting.getIteratorName());
++
++    List<AccumuloIteratorOption> iteratorOptions = AccumuloInputFormat.getIteratorOptions(job.getConfiguration());
++    assertEquals(2, iteratorOptions.size());
++    assertEquals("Count", iteratorOptions.get(0).getIteratorName());
++    assertEquals("Count", iteratorOptions.get(1).getIteratorName());
++    assertEquals("v1", iteratorOptions.get(0).getKey());
++    assertEquals("1", iteratorOptions.get(0).getValue());
++    assertEquals("junk", iteratorOptions.get(1).getKey());
++    assertEquals("\0omg:!\\xyzzy", iteratorOptions.get(1).getValue());
++  }
++
++  /**
++   * 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);
++    Job job = new Job();
++    AccumuloInputFormat.addIterator(job.getConfiguration(), someSetting);
++
++    final String rawConfigOpt = new AccumuloIteratorOption("iterator", key, value).toString();
++
++    assertEquals(rawConfigOpt, job.getConfiguration().get("AccumuloInputFormat.iterators.options"));
++
++    List<AccumuloIteratorOption> opts = AccumuloInputFormat.getIteratorOptions(job.getConfiguration());
++    assertEquals(1, opts.size());
++    assertEquals(opts.get(0).getKey(), key);
++    assertEquals(opts.get(0).getValue(), value);
++
++    someSetting.addOption(key + "2", value);
++    someSetting.setPriority(2);
++    someSetting.setName("it2");
++    AccumuloInputFormat.addIterator(job.getConfiguration(), someSetting);
++    opts = AccumuloInputFormat.getIteratorOptions(job.getConfiguration());
++    assertEquals(3, opts.size());
++    for (AccumuloIteratorOption opt : opts) {
++      assertEquals(opt.getKey().substring(0, key.length()), key);
++      assertEquals(opt.getValue(), value);
++    }
++  }
++
++  /**
++   * Test getting iterator settings for multiple iterators set
++   */
++  @SuppressWarnings("deprecation")
++  @Test
++  public void testGetIteratorSettings() {
++    JobContext job = new JobContext(new Configuration(), new JobID());
++
++    AccumuloInputFormat.setIterator(job, 1, "org.apache.accumulo.core.iterators.WholeRowIterator", "WholeRow");
++    AccumuloInputFormat.setIterator(job, 2, "org.apache.accumulo.core.iterators.VersioningIterator", "Versions");
++    AccumuloInputFormat.setIterator(job, 3, "org.apache.accumulo.core.iterators.CountingIterator", "Count");
++
++    List<AccumuloIterator> list = AccumuloInputFormat.getIterators(job);
++
++    // Check the list size
++    assertTrue(list.size() == 3);
++
++    // Walk the list and make sure our settings are correct
++    AccumuloIterator setting = list.get(0);
++    assertEquals(1, setting.getPriority());
++    assertEquals("org.apache.accumulo.core.iterators.WholeRowIterator", setting.getIteratorClass());
++    assertEquals("WholeRow", setting.getIteratorName());
++
++    setting = list.get(1);
++    assertEquals(2, setting.getPriority());
++    assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
++    assertEquals("Versions", setting.getIteratorName());
++
++    setting = list.get(2);
++    assertEquals(3, setting.getPriority());
++    assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
++    assertEquals("Count", setting.getIteratorName());
++
++  }
++
++  /**
++   * Check that the iterator options are getting stored in the Job conf correctly.
++   */
++  @SuppressWarnings("deprecation")
++  @Test
++  public void testSetIteratorOption() {
++    JobContext job = new JobContext(new Configuration(), new JobID());
++    AccumuloInputFormat.setIteratorOption(job, "someIterator", "aKey", "aValue");
++
++    Configuration conf = job.getConfiguration();
++    String options = conf.get("AccumuloInputFormat.iterators.options");
++    assertEquals(new String("someIterator:aKey:aValue"), options);
++  }
++
++  /**
++   * Test getting iterator options for multiple options set
++   */
++  @SuppressWarnings("deprecation")
++  @Test
++  public void testGetIteratorOption() {
++    JobContext job = new JobContext(new Configuration(), new JobID());
++
++    AccumuloInputFormat.setIteratorOption(job, "iterator1", "key1", "value1");
++    AccumuloInputFormat.setIteratorOption(job, "iterator2", "key2", "value2");
++    AccumuloInputFormat.setIteratorOption(job, "iterator3", "key3", "value3");
++
++    List<AccumuloIteratorOption> list = AccumuloInputFormat.getIteratorOptions(job);
++
++    // Check the list size
++    assertEquals(3, list.size());
++
++    // Walk the list and make sure all the options are correct
++    AccumuloIteratorOption option = list.get(0);
++    assertEquals("iterator1", option.getIteratorName());
++    assertEquals("key1", option.getKey());
++    assertEquals("value1", option.getValue());
++
++    option = list.get(1);
++    assertEquals("iterator2", option.getIteratorName());
++    assertEquals("key2", option.getKey());
++    assertEquals("value2", option.getValue());
++
++    option = list.get(2);
++    assertEquals("iterator3", option.getIteratorName());
++    assertEquals("key3", option.getKey());
++    assertEquals("value3", option.getValue());
++  }
++
++  @SuppressWarnings("deprecation")
++  @Test
++  public void testSetRegex() {
++    JobContext job = new JobContext(new Configuration(), new JobID());
++
++    String regex = ">\"*%<>\'\\";
++
++    AccumuloInputFormat.setRegex(job, RegexType.ROW, regex);
++
++    assertTrue(regex.equals(AccumuloInputFormat.getRegex(job, RegexType.ROW)));
++  }
++
++  static class TestMapper extends Mapper<Key,Value,Key,Value> {
++    Key key = null;
++    int count = 0;
++
++    @Override
++    protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
++      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));
++      key = new Key(k);
++      count++;
++    }
++  }
++
++  @Test
++  public void testMap() throws Exception {
++    MockInstance mockInstance = new MockInstance("testmapinstance");
++    Connector c = mockInstance.getConnector("root", new byte[] {});
++    c.tableOperations().create("testtable");
++    BatchWriter bw = c.createBatchWriter("testtable", 10000L, 1000L, 4);
++    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();
++
++    Job job = new Job(new Configuration());
++    job.setInputFormatClass(AccumuloInputFormat.class);
++    job.setMapperClass(TestMapper.class);
++    job.setNumReduceTasks(0);
++    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
++    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
++
++    AccumuloInputFormat input = new AccumuloInputFormat();
++    List<InputSplit> splits = input.getSplits(job);
++    assertEquals(splits.size(), 1);
++
++    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
++    for (InputSplit split : splits) {
++      RangeInputSplit risplit = (RangeInputSplit) split;
++      Assert.assertEquals("root", risplit.getUsername());
++      Assert.assertArrayEquals(new byte[0], risplit.getPassword());
++      Assert.assertEquals("testtable", risplit.getTable());
++      Assert.assertEquals(new Authorizations(), risplit.getAuths());
++      Assert.assertEquals("testmapinstance", risplit.getInstanceName());
++      
++      TaskAttemptID id = new TaskAttemptID();
++      TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
++      RecordReader<Key,Value> reader = input.createRecordReader(split, attempt);
++      Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, split);
++      reader.initialize(split, context);
++      mapper.run(context);
++    }
++  }
++
++  @Test
++  public void testSimple() throws Exception {
++    MockInstance mockInstance = new MockInstance("testmapinstance");
++    Connector c = mockInstance.getConnector("root", new byte[] {});
++    c.tableOperations().create("testtable2");
++    BatchWriter bw = c.createBatchWriter("testtable2", 10000L, 1000L, 4);
++    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();
++
++    JobContext job = new JobContext(new Configuration(), new JobID());
++    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable2", new Authorizations());
++    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
++    AccumuloInputFormat input = new AccumuloInputFormat();
++    RangeInputSplit ris = new RangeInputSplit();
++    TaskAttemptContext tac = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID());
++    RecordReader<Key,Value> rr = input.createRecordReader(ris, tac);
++    rr.initialize(ris, tac);
++
++    TestMapper mapper = new TestMapper();
++    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), tac.getTaskAttemptID(), rr, null, null, null, ris);
++    while (rr.nextKeyValue()) {
++      mapper.map(rr.getCurrentKey(), rr.getCurrentValue(), context);
++    }
++  }
++
++  @SuppressWarnings("deprecation")
++  @Test
++  public void testRegex() throws Exception {
++    MockInstance mockInstance = new MockInstance("testmapinstance");
++    Connector c = mockInstance.getConnector("root", new byte[] {});
++    c.tableOperations().create("testtable3");
++    BatchWriter bw = c.createBatchWriter("testtable3", 10000L, 1000L, 4);
++    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();
++
++    JobContext job = new JobContext(new Configuration(), new JobID());
++    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable3", new Authorizations());
++    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
++    final String regex = ".*1.*";
++    AccumuloInputFormat.setRegex(job, RegexType.ROW, regex);
++    AccumuloInputFormat input = new AccumuloInputFormat();
++    RangeInputSplit ris = new RangeInputSplit();
++    TaskAttemptContext tac = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID());
++    RecordReader<Key,Value> rr = input.createRecordReader(ris, tac);
++    rr.initialize(ris, tac);
++
++    Pattern p = Pattern.compile(regex);
++    while (rr.nextKeyValue()) {
++      Assert.assertTrue(p.matcher(rr.getCurrentKey().getRow().toString()).matches());
++    }
++  }
++
++  @SuppressWarnings("deprecation")
++  @Test
++  public void testCorrectRangeInputSplits() throws Exception {
++    JobContext job = new JobContext(new Configuration(), new JobID());
++
++    String username = "user", table = "table", rowRegex = "row.*", colfRegex = "colf.*", colqRegex = "colq.*";
++    String valRegex = "val.*", instance = "instance";
++    byte[] password = "password".getBytes();
++    Authorizations auths = new Authorizations("foo");
++    Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
++    boolean isolated = true, localIters = true;
++    int maxVersions = 5;
++    Level level = Level.WARN;
++
++    Instance inst = new MockInstance(instance);
++    Connector connector = inst.getConnector(username, password);
++    connector.tableOperations().create(table);
++
++    AccumuloInputFormat.setInputInfo(job, username, password, table, auths);
++    AccumuloInputFormat.setMockInstance(job, instance);
++    AccumuloInputFormat.setRegex(job, RegexType.ROW, rowRegex);
++    AccumuloInputFormat.setRegex(job, RegexType.COLUMN_FAMILY, colfRegex);
++    AccumuloInputFormat.setRegex(job, RegexType.COLUMN_QUALIFIER, colqRegex);
++    AccumuloInputFormat.setRegex(job, RegexType.VALUE, valRegex);
++    AccumuloInputFormat.setIsolated(job, isolated);
++    AccumuloInputFormat.setLocalIterators(job, localIters);
++    AccumuloInputFormat.setMaxVersions(job, maxVersions);
++    AccumuloInputFormat.fetchColumns(job, fetchColumns);
++    AccumuloInputFormat.setLogLevel(job, level);
++    
++    AccumuloInputFormat aif = new AccumuloInputFormat();
++    
++    List<InputSplit> splits = aif.getSplits(job);
++    
++    Assert.assertEquals(1, splits.size());
++    
++    InputSplit split = splits.get(0);
++    
++    Assert.assertEquals(RangeInputSplit.class, split.getClass());
++    
++    RangeInputSplit risplit = (RangeInputSplit) split;
++    
++    Assert.assertEquals(username, risplit.getUsername());
++    Assert.assertEquals(table, risplit.getTable());
++    Assert.assertArrayEquals(password, risplit.getPassword());
++    Assert.assertEquals(auths, risplit.getAuths());
++    Assert.assertEquals(instance, risplit.getInstanceName());
++    Assert.assertEquals(rowRegex, risplit.getRowRegex());
++    Assert.assertEquals(colfRegex, risplit.getColfamRegex());
++    Assert.assertEquals(colqRegex, risplit.getColqualRegex());
++    Assert.assertEquals(valRegex, risplit.getValueRegex());
++    Assert.assertEquals(isolated, risplit.isIsolatedScan());
++    Assert.assertEquals(localIters, risplit.usesLocalIterators());
++    Assert.assertEquals(maxVersions, risplit.getMaxVersions().intValue());
++    Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());
++    Assert.assertEquals(level, risplit.getLogLevel());
++  }
++
++  @Test
++  public void testPartialInputSplitDelegationToConfiguration() throws Exception {
++    MockInstance mockInstance = new MockInstance("testPartialInputSplitDelegationToConfiguration");
++    Connector c = mockInstance.getConnector("root", new byte[] {});
++    c.tableOperations().create("testtable");
++    BatchWriter bw = c.createBatchWriter("testtable", 10000L, 1000L, 4);
++    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();
++
++    Job job = new Job(new Configuration());
++    job.setInputFormatClass(AccumuloInputFormat.class);
++    job.setMapperClass(TestMapper.class);
++    job.setNumReduceTasks(0);
++    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
++    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testPartialInputSplitDelegationToConfiguration");
++
++    AccumuloInputFormat input = new AccumuloInputFormat();
++    List<InputSplit> splits = input.getSplits(job);
++    assertEquals(splits.size(), 1);
++
++    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
++    
++    RangeInputSplit emptySplit = new RangeInputSplit();
++    
++    // Using an empty split should fall back to the information in the Job's Configuration
++    TaskAttemptID id = new TaskAttemptID();
++    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
++    RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
++    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
++    reader.initialize(emptySplit, context);
++    mapper.run(context);
++  }
++
++  @Test(expected = IOException.class)
++  public void testPartialFailedInputSplitDelegationToConfiguration() throws Exception {
++    MockInstance mockInstance = new MockInstance("testPartialFailedInputSplitDelegationToConfiguration");
++    Connector c = mockInstance.getConnector("root", new byte[] {});
++    c.tableOperations().create("testtable");
++    BatchWriter bw = c.createBatchWriter("testtable", 10000L, 1000L, 4);
++    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();
++
++    Job job = new Job(new Configuration());
++    job.setInputFormatClass(AccumuloInputFormat.class);
++    job.setMapperClass(TestMapper.class);
++    job.setNumReduceTasks(0);
++    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
++    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testPartialFailedInputSplitDelegationToConfiguration");
++
++    AccumuloInputFormat input = new AccumuloInputFormat();
++    List<InputSplit> splits = input.getSplits(job);
++    assertEquals(splits.size(), 1);
++
++    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
++    
++    RangeInputSplit emptySplit = new RangeInputSplit();
++    emptySplit.setUsername("root");
++    emptySplit.setPassword("anythingelse".getBytes());
++    
++    // Using an empty split should fall back to the information in the Job's Configuration
++    TaskAttemptID id = new TaskAttemptID();
++    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
++    RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
++    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
++    reader.initialize(emptySplit, context);
++    mapper.run(context);
++  }
++}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/16a2e0f0/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
index 0000000,0000000..22fb6e1
new file mode 100644
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
@@@ -1,0 -1,0 +1,100 @@@
++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.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(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.getRange(), newSplit.getRange());
++    Assert.assertTrue(Arrays.equals(split.getLocations(), newSplit.getLocations()));
++  }
++
++
++
++  @Test
++  public void testAllFieldsWritable() throws IOException {
++    RangeInputSplit split = new RangeInputSplit(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.setMaxVersions(5);
++    split.setRowRegex("row");
++    split.setColfamRegex("colf");
++    split.setColqualRegex("colq");
++    split.setValueRegex("value");
++    split.setFetchedColumns(fetchedColumns);
++    split.setPassword("password".getBytes());
++    split.setUsername("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.getMaxVersions(), newSplit.getMaxVersions());
++    Assert.assertEquals(split.getRowRegex(), newSplit.getRowRegex());
++    Assert.assertEquals(split.getColfamRegex(), newSplit.getColfamRegex());
++    Assert.assertEquals(split.getColqualRegex(), newSplit.getColqualRegex());
++    Assert.assertEquals(split.getValueRegex(), newSplit.getValueRegex());
++    Assert.assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
++    Assert.assertEquals(new String(split.getPassword()), new String(newSplit.getPassword()));
++    Assert.assertEquals(split.getUsername(), newSplit.getUsername());
++    Assert.assertEquals(split.getInstanceName(), newSplit.getInstanceName());
++    Assert.assertEquals(split.isMockInstance(), newSplit.isMockInstance());
++    Assert.assertEquals(split.getZooKeepers(), newSplit.getZooKeepers());
++    Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel());
++  }
++  
++}


[16/19] git commit: Squashed commit of the following:

Posted by el...@apache.org.
Squashed commit of the following:

commit dfbe098fb650d1d1605ac28ff0b195e229ecb345
Author: Josh Elser <el...@apache.org>
Date:   Wed Nov 20 23:57:18 2013 -0500

    ACCUMULO-1843 Add in log4j Level to RangeInputSplit. Add more tests, notably ones that exercise delegation of the input
    split to the Configuration.

commit 38fdee9916edd938bea1642de5d4e5cf54a81596
Author: Josh Elser <el...@apache.org>
Date:   Fri Nov 8 17:47:57 2013 -0500

    ACCUMULO-1854 Fix up InputFormatBase to use the information stored on
    RangeInputSplit and fall back onto the Configuration.

commit 0e6d1aba7eacef357e0a17c67a453dd5b50a49dc
Author: Josh Elser <el...@apache.org>
Date:   Fri Nov 8 16:23:49 2013 -0500

    ACCUMULO-1854 Clean up constructors. Add a test.

commit 2f59f81f6e75f8a90ccfe3df00c6ad3f69174e0c
Author: Josh Elser <el...@apache.org>
Date:   Fri Nov 8 15:46:39 2013 -0500

    ACCUMULO-1854 Move RangeInputSplit into its own file and store all
    connection information into it.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/73114819
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/73114819
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/73114819

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 73114819e85714f56838e2bcf16bf9b5c6c6a397
Parents: 2640ea9
Author: Josh Elser <el...@apache.org>
Authored: Thu Nov 21 00:19:59 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Nov 22 15:17:57 2013 -0500

----------------------------------------------------------------------
 .../client/mapreduce/AccumuloInputFormat.java   |  11 +
 .../core/client/mapreduce/InputFormatBase.java  | 615 ++++++++++---------
 .../core/client/mapreduce/RangeInputSplit.java  | 493 +++++++++++++++
 .../mapreduce/AccumuloInputFormatTest.java      | 253 ++++++--
 .../mapreduce/AccumuloRowInputFormatTest.java   |   1 -
 .../client/mapreduce/RangeInputSplitTest.java   | 100 +++
 .../simple/filedata/ChunkInputFormatTest.java   |   4 +-
 7 files changed, 1126 insertions(+), 351 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/73114819/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
index 4de131f..c9a70eb 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.util.format.DefaultFormatter;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.log4j.Level;
 
 /**
  * This class allows MapReduce jobs to use Accumulo as the source of data. This input format provides keys and values of type Key and Value to the Map() and
@@ -44,6 +45,16 @@ public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
   @Override
   public RecordReader<Key,Value> createRecordReader(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
     log.setLevel(getLogLevel(context.getConfiguration()));
+    
+    // Override the log level from the configuration as if the RangeInputSplit has one it's the more correct one to use.
+    if (split instanceof RangeInputSplit) {
+      RangeInputSplit risplit = (RangeInputSplit) split;
+      Level level = risplit.getLogLevel();
+      if (null != level) {
+        log.setLevel(level);
+      }
+    }
+
     return new RecordReaderBase<Key,Value>() {
       @Override
       public boolean nextKeyValue() throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/73114819/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index 8e238f1..40e09a1 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -110,7 +110,7 @@ import org.apache.log4j.Logger;
 
 public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static final Logger log = Logger.getLogger(InputFormatBase.class);
-  
+
   private static final String PREFIX = AccumuloInputFormat.class.getSimpleName();
   private static final String INPUT_INFO_HAS_BEEN_SET = PREFIX + ".configured";
   private static final String INSTANCE_HAS_BEEN_SET = PREFIX + ".instanceConfigured";
@@ -118,34 +118,34 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   private static final String PASSWORD = PREFIX + ".password";
   private static final String TABLE_NAME = PREFIX + ".tablename";
   private static final String AUTHORIZATIONS = PREFIX + ".authorizations";
-  
+
   private static final String INSTANCE_NAME = PREFIX + ".instanceName";
   private static final String ZOOKEEPERS = PREFIX + ".zooKeepers";
   private static final String MOCK = ".useMockInstance";
-  
+
   private static final String RANGES = PREFIX + ".ranges";
   private static final String AUTO_ADJUST_RANGES = PREFIX + ".ranges.autoAdjust";
-  
+
   private static final String ROW_REGEX = PREFIX + ".regex.row";
   private static final String COLUMN_FAMILY_REGEX = PREFIX + ".regex.cf";
   private static final String COLUMN_QUALIFIER_REGEX = PREFIX + ".regex.cq";
   private static final String VALUE_REGEX = PREFIX + ".regex.value";
-  
+
   private static final String COLUMNS = PREFIX + ".columns";
   private static final String LOGLEVEL = PREFIX + ".loglevel";
-  
+
   private static final String ISOLATED = PREFIX + ".isolated";
-  
+
   private static final String LOCAL_ITERATORS = PREFIX + ".localiters";
-  
+
   // Used to specify the maximum # of versions of an Accumulo cell value to return
   private static final String MAX_VERSIONS = PREFIX + ".maxVersions";
-  
+
   // Used for specifying the iterators to be applied
   private static final String ITERATORS = PREFIX + ".iterators";
   private static final String ITERATORS_OPTIONS = PREFIX + ".iterators.options";
   private static final String ITERATORS_DELIM = ",";
-  
+
   private static final String READ_OFFLINE = PREFIX + ".read.offline";
 
   /**
@@ -154,7 +154,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIsolated(JobContext job, boolean enable) {
     setIsolated(job.getConfiguration(), enable);
   }
-  
+
   /**
    * Enable or disable use of the {@link IsolatedScanner} in this configuration object. By default it is not enabled.
    * 
@@ -166,14 +166,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIsolated(Configuration conf, boolean enable) {
     conf.setBoolean(ISOLATED, enable);
   }
-  
+
   /**
    * @deprecated Use {@link #setLocalIterators(Configuration,boolean)} instead
    */
   public static void setLocalIterators(JobContext job, boolean enable) {
     setLocalIterators(job.getConfiguration(), enable);
   }
-  
+
   /**
    * Enable or disable use of the {@link ClientSideIteratorScanner} in this Configuration object. By default it is not enabled.
    * 
@@ -185,14 +185,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setLocalIterators(Configuration conf, boolean enable) {
     conf.setBoolean(LOCAL_ITERATORS, enable);
   }
-  
+
   /**
    * @deprecated Use {@link #setInputInfo(Configuration,String,byte[],String,Authorizations)} instead
    */
   public static void setInputInfo(JobContext job, String user, byte[] passwd, String table, Authorizations auths) {
     setInputInfo(job.getConfiguration(), user, passwd, table, auths);
   }
-  
+
   /**
    * Initialize the user, table, and authorization information for the configuration object that will be used with an Accumulo InputFormat.
    * 
@@ -211,7 +211,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     if (conf.getBoolean(INPUT_INFO_HAS_BEEN_SET, false))
       throw new IllegalStateException("Input info can only be set once per job");
     conf.setBoolean(INPUT_INFO_HAS_BEEN_SET, true);
-    
+
     ArgumentChecker.notNull(user, passwd, table);
     conf.set(USERNAME, user);
     conf.set(PASSWORD, new String(Base64.encodeBase64(passwd)));
@@ -219,14 +219,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     if (auths != null && !auths.isEmpty())
       conf.set(AUTHORIZATIONS, auths.serialize());
   }
-  
+
   /**
    * @deprecated Use {@link #setZooKeeperInstance(Configuration,String,String)} instead
    */
   public static void setZooKeeperInstance(JobContext job, String instanceName, String zooKeepers) {
     setZooKeeperInstance(job.getConfiguration(), instanceName, zooKeepers);
   }
-  
+
   /**
    * Configure a {@link ZooKeeperInstance} for this configuration object.
    * 
@@ -241,19 +241,19 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     if (conf.getBoolean(INSTANCE_HAS_BEEN_SET, false))
       throw new IllegalStateException("Instance info can only be set once per job");
     conf.setBoolean(INSTANCE_HAS_BEEN_SET, true);
-    
+
     ArgumentChecker.notNull(instanceName, zooKeepers);
     conf.set(INSTANCE_NAME, instanceName);
     conf.set(ZOOKEEPERS, zooKeepers);
   }
-  
+
   /**
    * @deprecated Use {@link #setMockInstance(Configuration,String)} instead
    */
   public static void setMockInstance(JobContext job, String instanceName) {
     setMockInstance(job.getConfiguration(), instanceName);
   }
-  
+
   /**
    * Configure a {@link MockInstance} for this configuration object.
    * 
@@ -267,14 +267,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     conf.setBoolean(MOCK, true);
     conf.set(INSTANCE_NAME, instanceName);
   }
-  
+
   /**
    * @deprecated Use {@link #setRanges(Configuration,Collection)} instead
    */
   public static void setRanges(JobContext job, Collection<Range> ranges) {
     setRanges(job.getConfiguration(), ranges);
   }
-  
+
   /**
    * Set the ranges to map over for this configuration object.
    * 
@@ -297,14 +297,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     conf.setStrings(RANGES, rangeStrings.toArray(new String[0]));
   }
-  
+
   /**
    * @deprecated Use {@link #disableAutoAdjustRanges(Configuration)} instead
    */
   public static void disableAutoAdjustRanges(JobContext job) {
     disableAutoAdjustRanges(job.getConfiguration());
   }
-  
+
   /**
    * Disables the adjustment of ranges for this configuration object. By default, overlapping ranges will be merged and ranges will be fit to existing tablet
    * boundaries. Disabling this adjustment will cause there to be exactly one mapper per range set using {@link #setRanges(Configuration, Collection)}.
@@ -315,14 +315,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void disableAutoAdjustRanges(Configuration conf) {
     conf.setBoolean(AUTO_ADJUST_RANGES, false);
   }
-  
+
   /**
    * @deprecated since 1.4 use {@link org.apache.accumulo.core.iterators.user.RegExFilter} and {@link #addIterator(Configuration, IteratorSetting)}
    */
   public static enum RegexType {
     ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VALUE
   }
-  
+
   /**
    * @deprecated since 1.4 use {@link #addIterator(Configuration, IteratorSetting)}
    * @see org.apache.accumulo.core.iterators.user.RegExFilter#setRegexs(IteratorSetting, String, String, String, String, boolean)
@@ -356,14 +356,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * @deprecated Use {@link #setMaxVersions(Configuration,int)} instead
    */
   public static void setMaxVersions(JobContext job, int maxVersions) throws IOException {
     setMaxVersions(job.getConfiguration(), maxVersions);
   }
-  
+
   /**
    * Sets the max # of values that may be returned for an individual Accumulo cell. By default, applied before all other Accumulo iterators (highest priority)
    * leveraged in the scan by the record reader. To adjust priority use setIterator() & setIteratorOptions() w/ the VersioningIterator type explicitly.
@@ -380,7 +380,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       throw new IOException("Invalid maxVersions: " + maxVersions + ".  Must be >= 1");
     conf.setInt(MAX_VERSIONS, maxVersions);
   }
-  
+
   /**
    * <p>
    * Enable reading offline tables. This will make the map reduce job directly read the tables files. If the table is not offline, then the job will fail. If
@@ -407,18 +407,18 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @param scanOff
    *          pass true to read offline tables
    */
-  
+
   public static void setScanOffline(Configuration conf, boolean scanOff) {
     conf.setBoolean(READ_OFFLINE, scanOff);
   }
-  
+
   /**
    * @deprecated Use {@link #fetchColumns(Configuration,Collection)} instead
    */
   public static void fetchColumns(JobContext job, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
     fetchColumns(job.getConfiguration(), columnFamilyColumnQualifierPairs);
   }
-  
+
   /**
    * Restricts the columns that will be mapped over for this configuration object.
    * 
@@ -429,27 +429,33 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    *          selected. An empty set is the default and is equivalent to scanning the all columns.
    */
   public static void fetchColumns(Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
+    String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
+    conf.setStrings(COLUMNS, columnStrings);
+  }
+
+  public static String[] serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
     ArgumentChecker.notNull(columnFamilyColumnQualifierPairs);
     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())));
       if (column.getSecond() != null)
         col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())));
       columnStrings.add(col);
     }
-    conf.setStrings(COLUMNS, columnStrings.toArray(new String[0]));
+
+    return columnStrings.toArray(new String[0]);
   }
-  
+
   /**
    * @deprecated Use {@link #setLogLevel(Configuration,Level)} instead
    */
   public static void setLogLevel(JobContext job, Level level) {
     setLogLevel(job.getConfiguration(), level);
   }
-  
+
   /**
    * Sets the log level for this configuration object.
    * 
@@ -463,14 +469,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     log.setLevel(level);
     conf.setInt(LOGLEVEL, level.toInt());
   }
-  
+
   /**
    * @deprecated Use {@link #addIterator(Configuration,IteratorSetting)} instead
    */
   public static void addIterator(JobContext job, IteratorSetting cfg) {
     addIterator(job.getConfiguration(), cfg);
   }
-  
+
   /**
    * Encode an iterator on the input for this configuration object.
    * 
@@ -482,7 +488,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void addIterator(Configuration conf, IteratorSetting cfg) {
     // First check to see if anything has been set already
     String iterators = conf.get(ITERATORS);
-    
+
     // No iterators specified yet, create a new string
     if (iterators == null || iterators.isEmpty()) {
       iterators = new AccumuloIterator(cfg.getPriority(), cfg.getIteratorClass(), cfg.getName()).toString();
@@ -495,9 +501,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     for (Entry<String,String> entry : cfg.getOptions().entrySet()) {
       if (entry.getValue() == null)
         continue;
-      
+
       String iteratorOptions = conf.get(ITERATORS_OPTIONS);
-      
+
       // No options specified yet, create a new string
       if (iteratorOptions == null || iteratorOptions.isEmpty()) {
         iteratorOptions = new AccumuloIteratorOption(cfg.getName(), entry.getKey(), entry.getValue()).toString();
@@ -505,12 +511,12 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         // append the next option & reset
         iteratorOptions = iteratorOptions.concat(ITERATORS_DELIM + new AccumuloIteratorOption(cfg.getName(), entry.getKey(), entry.getValue()));
       }
-      
+
       // Store the options w/ the job
       conf.set(ITERATORS_OPTIONS, iteratorOptions);
     }
   }
-  
+
   /**
    * Specify an Accumulo iterator type to manage the behavior of the underlying table scan this InputFormat's RecordReader will conduct, w/ priority dictating
    * the order in which specified iterators are applied. Repeat calls to specify multiple iterators are allowed.
@@ -529,7 +535,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIterator(JobContext job, int priority, String iteratorClass, String iteratorName) {
     // First check to see if anything has been set already
     String iterators = job.getConfiguration().get(ITERATORS);
-    
+
     // No iterators specified yet, create a new string
     if (iterators == null || iterators.isEmpty()) {
       iterators = new AccumuloIterator(priority, iteratorClass, iteratorName).toString();
@@ -539,9 +545,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     // Store the iterators w/ the job
     job.getConfiguration().set(ITERATORS, iterators);
-    
+
   }
-  
+
   /**
    * Specify an option for a named Accumulo iterator, further specifying that iterator's behavior.
    * 
@@ -559,9 +565,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIteratorOption(JobContext job, String iteratorName, String key, String value) {
     if (iteratorName == null || key == null || value == null)
       return;
-    
+
     String iteratorOptions = job.getConfiguration().get(ITERATORS_OPTIONS);
-    
+
     // No options specified yet, create a new string
     if (iteratorOptions == null || iteratorOptions.isEmpty()) {
       iteratorOptions = new AccumuloIteratorOption(iteratorName, key, value).toString();
@@ -569,18 +575,18 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       // append the next option & reset
       iteratorOptions = iteratorOptions.concat(ITERATORS_DELIM + new AccumuloIteratorOption(iteratorName, key, value));
     }
-    
+
     // Store the options w/ the job
     job.getConfiguration().set(ITERATORS_OPTIONS, iteratorOptions);
   }
-  
+
   /**
    * @deprecated Use {@link #isIsolated(Configuration)} instead
    */
   protected static boolean isIsolated(JobContext job) {
     return isIsolated(job.getConfiguration());
   }
-  
+
   /**
    * Determines whether a configuration has isolation enabled.
    * 
@@ -592,14 +598,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean isIsolated(Configuration conf) {
     return conf.getBoolean(ISOLATED, false);
   }
-  
+
   /**
    * @deprecated Use {@link #usesLocalIterators(Configuration)} instead
    */
   protected static boolean usesLocalIterators(JobContext job) {
     return usesLocalIterators(job.getConfiguration());
   }
-  
+
   /**
    * Determines whether a configuration uses local iterators.
    * 
@@ -611,14 +617,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean usesLocalIterators(Configuration conf) {
     return conf.getBoolean(LOCAL_ITERATORS, false);
   }
-  
+
   /**
    * @deprecated Use {@link #getUsername(Configuration)} instead
    */
   protected static String getUsername(JobContext job) {
     return getUsername(job.getConfiguration());
   }
-  
+
   /**
    * Gets the user name from the configuration.
    * 
@@ -630,7 +636,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static String getUsername(Configuration conf) {
     return conf.get(USERNAME);
   }
-  
+
   /**
    * WARNING: The password 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.
@@ -640,7 +646,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static byte[] getPassword(JobContext job) {
     return getPassword(job.getConfiguration());
   }
-  
+
   /**
    * Gets the password from the configuration. WARNING: The password 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.
@@ -653,14 +659,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static byte[] getPassword(Configuration conf) {
     return Base64.decodeBase64(conf.get(PASSWORD, "").getBytes());
   }
-  
+
   /**
    * @deprecated Use {@link #getTablename(Configuration)} instead
    */
   protected static String getTablename(JobContext job) {
     return getTablename(job.getConfiguration());
   }
-  
+
   /**
    * Gets the table name from the configuration.
    * 
@@ -672,14 +678,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static String getTablename(Configuration conf) {
     return conf.get(TABLE_NAME);
   }
-  
+
   /**
    * @deprecated Use {@link #getAuthorizations(Configuration)} instead
    */
   protected static Authorizations getAuthorizations(JobContext job) {
     return getAuthorizations(job.getConfiguration());
   }
-  
+
   /**
    * Gets the authorizations to set for the scans from the configuration.
    * 
@@ -692,14 +698,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     String authString = conf.get(AUTHORIZATIONS);
     return authString == null ? Constants.NO_AUTHS : new Authorizations(authString.split(","));
   }
-  
+
   /**
    * @deprecated Use {@link #getInstance(Configuration)} instead
    */
   protected static Instance getInstance(JobContext job) {
     return getInstance(job.getConfiguration());
   }
-  
+
   /**
    * Initializes an Accumulo {@link Instance} based on the configuration.
    * 
@@ -714,14 +720,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       return new MockInstance(conf.get(INSTANCE_NAME));
     return new ZooKeeperInstance(conf.get(INSTANCE_NAME), conf.get(ZOOKEEPERS));
   }
-  
+
   /**
    * @deprecated Use {@link #getTabletLocator(Configuration)} instead
    */
   protected static TabletLocator getTabletLocator(JobContext job) throws TableNotFoundException {
     return getTabletLocator(job.getConfiguration());
   }
-  
+
   /**
    * Initializes an Accumulo {@link TabletLocator} based on the configuration.
    * 
@@ -741,14 +747,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     return TabletLocator.getInstance(instance, new AuthInfo(username, ByteBuffer.wrap(password), instance.getInstanceID()),
         new Text(Tables.getTableId(instance, tableName)));
   }
-  
+
   /**
    * @deprecated Use {@link #getRanges(Configuration)} instead
    */
   protected static List<Range> getRanges(JobContext job) throws IOException {
     return getRanges(job.getConfiguration());
   }
-  
+
   /**
    * Gets the ranges to scan over from a configuration object.
    * 
@@ -769,7 +775,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return ranges;
   }
-  
+
   /**
    * @deprecated since 1.4 use {@link org.apache.accumulo.core.iterators.user.RegExFilter} and {@link #addIterator(Configuration, IteratorSetting)}
    * @see #setRegex(JobContext, RegexType, String)
@@ -802,14 +808,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * @deprecated Use {@link #getFetchedColumns(Configuration)} instead
    */
   protected static Set<Pair<Text,Text>> getFetchedColumns(JobContext job) {
     return getFetchedColumns(job.getConfiguration());
   }
-  
+
   /**
    * Gets the columns to be mapped over from this configuration object.
    * 
@@ -819,8 +825,19 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @see #fetchColumns(Configuration, Collection)
    */
   protected static Set<Pair<Text,Text>> getFetchedColumns(Configuration conf) {
+    ArgumentChecker.notNull(conf);
+
+    return deserializeFetchedColumns(conf.getStrings(COLUMNS));
+  }
+
+  public static Set<Pair<Text,Text>> deserializeFetchedColumns(String[] serialized) {
     Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
-    for (String col : conf.getStringCollection(COLUMNS)) {
+
+    if (null == serialized) {
+      return columns;
+    }
+
+    for (String col : serialized) {
       int idx = col.indexOf(":");
       Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes()) : Base64.decodeBase64(col.substring(0, idx).getBytes()));
       Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes()));
@@ -828,14 +845,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return columns;
   }
-  
+
   /**
    * @deprecated Use {@link #getAutoAdjustRanges(Configuration)} instead
    */
   protected static boolean getAutoAdjustRanges(JobContext job) {
     return getAutoAdjustRanges(job.getConfiguration());
   }
-  
+
   /**
    * Determines whether a configuration has auto-adjust ranges enabled.
    * 
@@ -847,14 +864,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean getAutoAdjustRanges(Configuration conf) {
     return conf.getBoolean(AUTO_ADJUST_RANGES, true);
   }
-  
+
   /**
    * @deprecated Use {@link #getLogLevel(Configuration)} instead
    */
   protected static Level getLogLevel(JobContext job) {
     return getLogLevel(job.getConfiguration());
   }
-  
+
   /**
    * Gets the log level from this configuration.
    * 
@@ -866,7 +883,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static Level getLogLevel(Configuration conf) {
     return Level.toLevel(conf.getInt(LOGLEVEL, Level.INFO.toInt()));
   }
-  
+
   // InputFormat doesn't have the equivalent of OutputFormat's
   // checkOutputSpecs(JobContext job)
   /**
@@ -875,7 +892,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static void validateOptions(JobContext job) throws IOException {
     validateOptions(job.getConfiguration());
   }
-  
+
   // InputFormat doesn't have the equivalent of OutputFormat's
   // checkOutputSpecs(JobContext job)
   /**
@@ -898,7 +915,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new IOException("Unable to authenticate user");
       if (!c.securityOperations().hasTablePermission(getUsername(conf), getTablename(conf), TablePermission.READ))
         throw new IOException("Unable to access table");
-      
+
       if (!usesLocalIterators(conf)) {
         // validate that any scan-time iterators can be loaded by the the tablet servers
         for (AccumuloIterator iter : getIterators(conf)) {
@@ -906,21 +923,21 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
             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);
     }
   }
-  
+
   /**
    * @deprecated Use {@link #getMaxVersions(Configuration)} instead
    */
   protected static int getMaxVersions(JobContext job) {
     return getMaxVersions(job.getConfiguration());
   }
-  
+
   /**
    * Gets the maxVersions to use for the {@link VersioningIterator} from this configuration.
    * 
@@ -932,7 +949,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static int getMaxVersions(Configuration conf) {
     return conf.getInt(MAX_VERSIONS, -1);
   }
-  
+
   protected static boolean isOfflineScan(Configuration conf) {
     return conf.getBoolean(READ_OFFLINE, false);
   }
@@ -945,7 +962,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static List<AccumuloIterator> getIterators(JobContext job) {
     return getIterators(job.getConfiguration());
   }
-  
+
   /**
    * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
    * 
@@ -955,13 +972,13 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @see #addIterator(Configuration, IteratorSetting)
    */
   protected static List<AccumuloIterator> getIterators(Configuration conf) {
-    
+
     String iterators = conf.get(ITERATORS);
-    
+
     // If no iterators are present, return an empty list
     if (iterators == null || iterators.isEmpty())
       return new ArrayList<AccumuloIterator>();
-    
+
     // Compose the set of iterators encoded in the job configuration
     StringTokenizer tokens = new StringTokenizer(conf.get(ITERATORS), ITERATORS_DELIM);
     List<AccumuloIterator> list = new ArrayList<AccumuloIterator>();
@@ -971,14 +988,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return list;
   }
-  
+
   /**
    * @deprecated Use {@link #getIteratorOptions(Configuration)} instead
    */
   protected static List<AccumuloIteratorOption> getIteratorOptions(JobContext job) {
     return getIteratorOptions(job.getConfiguration());
   }
-  
+
   /**
    * Gets a list of the iterator options specified on this configuration.
    * 
@@ -989,11 +1006,11 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    */
   protected static List<AccumuloIteratorOption> getIteratorOptions(Configuration conf) {
     String iteratorOptions = conf.get(ITERATORS_OPTIONS);
-    
+
     // If no options are present, return an empty list
     if (iteratorOptions == null || iteratorOptions.isEmpty())
       return new ArrayList<AccumuloIteratorOption>();
-    
+
     // Compose the set of options encoded in the job configuration
     StringTokenizer tokens = new StringTokenizer(conf.get(ITERATORS_OPTIONS), ITERATORS_DELIM);
     List<AccumuloIteratorOption> list = new ArrayList<AccumuloIteratorOption>();
@@ -1003,13 +1020,13 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return list;
   }
-  
+
   protected abstract static class RecordReaderBase<K,V> extends RecordReader<K,V> {
     protected long numKeysRead;
     protected Iterator<Entry<Key,Value>> scannerIterator;
     private boolean scannerRegexEnabled = false;
     protected RangeInputSplit split;
-    
+
     /**
      * @deprecated since 1.4, configure {@link org.apache.accumulo.core.iterators.user.RegExFilter} instead.
      */
@@ -1024,7 +1041,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         log.info("Setting " + methodName + " to " + regex);
       }
     }
-    
+
     /**
      * @deprecated since 1.4, configure {@link org.apache.accumulo.core.iterators.user.RegExFilter} instead.
      */
@@ -1039,15 +1056,16 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new AccumuloException("Can't set up regex for scanner");
       }
     }
-    
+
     // Apply the configured iterators from the job to the scanner
     /**
      * @deprecated Use {@link #setupIterators(Configuration,Scanner)} instead
      */
-    protected void setupIterators(TaskAttemptContext attempt, Scanner scanner) throws AccumuloException {
-      setupIterators(attempt.getConfiguration(), scanner);
+    protected void setupIterators(TaskAttemptContext attempt, Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
+        throws AccumuloException {
+      setupIterators(attempt.getConfiguration(), scanner, iterators, options);
     }
-    
+
     /**
      * Apply the configured iterators from the configuration to the scanner.
      * 
@@ -1057,10 +1075,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
      *          the scanner to configure
      * @throws AccumuloException
      */
-    protected void setupIterators(Configuration conf, Scanner scanner) throws AccumuloException {
-      List<AccumuloIterator> iterators = getIterators(conf);
-      List<AccumuloIteratorOption> options = getIteratorOptions(conf);
-      
+    protected void setupIterators(Configuration conf, Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
+        throws AccumuloException {
+
       Map<String,IteratorSetting> scanIterators = new HashMap<String,IteratorSetting>();
       for (AccumuloIterator iterator : iterators) {
         scanIterators.put(iterator.getIteratorName(), new IteratorSetting(iterator.getPriority(), iterator.getIteratorName(), iterator.getIteratorClass()));
@@ -1072,14 +1089,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         scanner.addScanIterator(scanIterators.get(iterator.getIteratorName()));
       }
     }
-    
+
     /**
      * @deprecated Use {@link #setupMaxVersions(Configuration,Scanner)} instead
      */
-    protected void setupMaxVersions(TaskAttemptContext attempt, Scanner scanner) {
-      setupMaxVersions(attempt.getConfiguration(), scanner);
+    protected void setupMaxVersions(TaskAttemptContext attempt, Scanner scanner, int maxVersions) {
+      setupMaxVersions(attempt.getConfiguration(), scanner, maxVersions);
     }
-    
+
     /**
      * If maxVersions has been set, configure a {@link VersioningIterator} at priority 0 for this scanner.
      * 
@@ -1088,8 +1105,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
      * @param scanner
      *          the scanner to configure
      */
-    protected void setupMaxVersions(Configuration conf, Scanner scanner) {
-      int maxVersions = getMaxVersions(conf);
+    protected void setupMaxVersions(Configuration conf, Scanner scanner, int maxVersions) {
       // Check to make sure its a legit value
       if (maxVersions >= 1) {
         IteratorSetting vers = new IteratorSetting(0, "vers", VersioningIterator.class);
@@ -1097,54 +1113,128 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         scanner.addScanIterator(vers);
       }
     }
-    
+
     /**
      * Initialize a scanner over the given input split using this task attempt configuration.
      */
     public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
       Scanner scanner;
       split = (RangeInputSplit) inSplit;
-      log.debug("Initializing input split: " + split.range);
+      log.debug("Initializing input split: " + split.getRange());
       Configuration conf = attempt.getConfiguration();
-      Instance instance = getInstance(conf);
-      String user = getUsername(conf);
-      byte[] password = getPassword(conf);
-      Authorizations authorizations = getAuthorizations(conf);
+
+      Instance instance = split.getInstance();
+      if (null == instance) {
+        instance = getInstance(conf);
+      }
+
+      String user = split.getUsername();
+      if (null == user) {
+        user = getUsername(conf);
+      }
+
+      byte[] password = split.getPassword();
+      if (null == password) {
+        password = getPassword(conf);
+      }
+
+      Authorizations authorizations = split.getAuths();
+      if (null == authorizations) {
+        authorizations = getAuthorizations(conf);
+      }
+
+      String table = split.getTable();
+      if (null == table) {
+        table = getTablename(conf);
+      }
+      
+      Boolean isOffline = split.isOffline();
+      if (null == isOffline) {
+        isOffline = isOfflineScan(conf);
+      }
+
+      Boolean isIsolated = split.isIsolatedScan();
+      if (null == isIsolated) {
+        isIsolated = isIsolated(conf);
+      }
+
+      Boolean usesLocalIterators = split.usesLocalIterators();
+      if (null == usesLocalIterators) {
+        usesLocalIterators = usesLocalIterators(conf);
+      }
+
+      String rowRegex = split.getRowRegex();
+      if (null == rowRegex) {
+        rowRegex = conf.get(ROW_REGEX);
+      }
+
+      String colfRegex = split.getColfamRegex();
+      if (null == colfRegex) {
+        colfRegex = conf.get(COLUMN_FAMILY_REGEX);
+      }
+
+      String colqRegex = split.getColqualRegex();
+      if (null == colqRegex) {
+        colqRegex = conf.get(COLUMN_QUALIFIER_REGEX);
+      }
+
+      String valueRegex = split.getValueRegex();
+      if (null == valueRegex) {
+        valueRegex = conf.get(VALUE_REGEX);
+      }
+
+      Integer maxVersions = split.getMaxVersions();
+      if (null == maxVersions) {
+        maxVersions = getMaxVersions(conf);
+      }
+      
+      List<AccumuloIterator> iterators = split.getIterators();
+      if (null == iterators) {
+        iterators = getIterators(conf);
+      }
+      
+      List<AccumuloIteratorOption> options = split.getOptions();
+      if (null == options) {
+        options = getIteratorOptions(conf);
+      }
       
+      Set<Pair<Text,Text>> columns = split.getFetchedColumns();
+      if (null == columns) {
+        columns = getFetchedColumns(conf);
+      }
+
       try {
         log.debug("Creating connector with user: " + user);
         Connector conn = instance.getConnector(user, password);
-        log.debug("Creating scanner for table: " + getTablename(conf));
+        log.debug("Creating scanner for table: " + table);
         log.debug("Authorizations are: " + authorizations);
-        if (isOfflineScan(conf)) {
-          scanner = new OfflineScanner(instance, new AuthInfo(user, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(instance,
-              getTablename(conf)), authorizations);
+        if (isOffline) {
+          scanner = new OfflineScanner(instance, new AuthInfo(user, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(instance, table),
+              authorizations);
         } else {
-          scanner = conn.createScanner(getTablename(conf), authorizations);
+          scanner = conn.createScanner(table, authorizations);
         }
-        if (isIsolated(conf)) {
+        if (isIsolated) {
           log.info("Creating isolated scanner");
           scanner = new IsolatedScanner(scanner);
         }
-        if (usesLocalIterators(conf)) {
+        if (usesLocalIterators) {
           log.info("Using local iterators");
           scanner = new ClientSideIteratorScanner(scanner);
         }
-        setupMaxVersions(conf, scanner);
-        if (conf.get(ROW_REGEX) != null || conf.get(COLUMN_FAMILY_REGEX) != null || conf.get(COLUMN_QUALIFIER_REGEX) != null ||
-            conf.get(VALUE_REGEX) != null) {
+        setupMaxVersions(conf, scanner, maxVersions);
+        if (rowRegex != null || colfRegex != null || colqRegex != null || valueRegex != null) {
           IteratorSetting is = new IteratorSetting(50, RegExFilter.class);
-          RegExFilter.setRegexs(is, conf.get(ROW_REGEX), conf.get(COLUMN_FAMILY_REGEX), conf.get(COLUMN_QUALIFIER_REGEX),
-            conf.get(VALUE_REGEX), false);
+          RegExFilter.setRegexs(is, rowRegex, colfRegex, colqRegex, valueRegex, false);
           scanner.addScanIterator(is);
         }
-        setupIterators(conf, scanner);
+        setupIterators(conf, scanner, iterators, options);
       } catch (Exception e) {
         throw new IOException(e);
       }
-      
+
       // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : getFetchedColumns(conf)) {
+      for (Pair<Text,Text> c : columns) {
         if (c.getSecond() != null) {
           log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
           scanner.fetchColumn(c.getFirst(), c.getSecond());
@@ -1153,48 +1243,48 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           scanner.fetchColumnFamily(c.getFirst());
         }
       }
-      
-      scanner.setRange(split.range);
-      
+
+      scanner.setRange(split.getRange());
+
       numKeysRead = 0;
-      
+
       // do this last after setting all scanner options
       scannerIterator = scanner.iterator();
     }
-    
+
     public void close() {}
-    
+
     public float getProgress() throws IOException {
       if (numKeysRead > 0 && currentKey == null)
         return 1.0f;
       return split.getProgress(currentKey);
     }
-    
+
     protected K currentK = null;
     protected V currentV = null;
     protected Key currentKey = null;
     protected Value currentValue = null;
-    
+
     @Override
     public K getCurrentKey() throws IOException, InterruptedException {
       return currentK;
     }
-    
+
     @Override
     public V getCurrentValue() throws IOException, InterruptedException {
       return currentV;
     }
   }
-  
+
   Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext job, String tableName, List<Range> ranges) throws TableNotFoundException,
       AccumuloException, AccumuloSecurityException {
-    
+
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 
     Instance instance = getInstance(job.getConfiguration());
     Connector conn = instance.getConnector(getUsername(job.getConfiguration()), getPassword(job.getConfiguration()));
     String tableId = Tables.getTableId(instance, tableName);
-    
+
     if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
       Tables.clearCache(instance);
       if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
@@ -1204,12 +1294,12 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
 
     for (Range range : ranges) {
       Text startRow;
-      
+
       if (range.getStartKey() != null)
         startRow = range.getStartKey().getRow();
       else
         startRow = new Text();
-      
+
       Range metadataRange = new Range(new KeyExtent(new Text(tableId), startRow, null).getMetadataEntry(), true, null, false);
       Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
       ColumnFQ.fetch(scanner, Constants.METADATA_PREV_ROW_COLUMN);
@@ -1217,9 +1307,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
       scanner.fetchColumnFamily(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY);
       scanner.setRange(metadataRange);
-      
+
       RowIterator rowIter = new RowIterator(scanner);
-      
+
       // TODO check that extents match prev extent
 
       KeyExtent lastExtent = null;
@@ -1229,15 +1319,15 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         String last = "";
         KeyExtent extent = null;
         String location = null;
-        
+
         while (row.hasNext()) {
           Entry<Key,Value> entry = row.next();
           Key key = entry.getKey();
-          
+
           if (key.getColumnFamily().equals(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY)) {
             last = entry.getValue().toString();
           }
-          
+
           if (key.getColumnFamily().equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY)
               || key.getColumnFamily().equals(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY)) {
             location = entry.getValue().toString();
@@ -1246,9 +1336,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           if (Constants.METADATA_PREV_ROW_COLUMN.hasColumns(key)) {
             extent = new KeyExtent(key.getRow(), entry.getValue());
           }
-          
+
         }
-        
+
         if (location != null)
           return null;
 
@@ -1265,24 +1355,24 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           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;
   }
 
@@ -1290,18 +1380,35 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * Read the metadata table to get tablets and match up ranges to them.
    */
   public List<InputSplit> getSplits(JobContext job) throws IOException {
-    log.setLevel(getLogLevel(job.getConfiguration()));
-    validateOptions(job.getConfiguration());
-    
-    String tableName = getTablename(job.getConfiguration());
-    boolean autoAdjust = getAutoAdjustRanges(job.getConfiguration());
-    List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(job.getConfiguration())) : getRanges(job.getConfiguration());
-    
+    Configuration conf = job.getConfiguration();
+
+    log.setLevel(getLogLevel(conf));
+    validateOptions(conf);
+
+    String tableName = getTablename(conf);
+    boolean autoAdjust = getAutoAdjustRanges(conf);
+    List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(conf)) : getRanges(conf);
+    boolean offline = isOfflineScan(conf);
+    boolean isolated = isIsolated(conf);
+    boolean localIterators = usesLocalIterators(conf);
+    boolean mockInstance = conf.getBoolean(MOCK, false);
+    int maxVersions = getMaxVersions(conf);
+    String rowRegex = conf.get(ROW_REGEX), colfamRegex = conf.get(COLUMN_FAMILY_REGEX), colqualRegex = conf.get(COLUMN_QUALIFIER_REGEX), valueRegex = conf
+        .get(VALUE_REGEX);
+    Set<Pair<Text,Text>> fetchedColumns = getFetchedColumns(conf);
+    Authorizations auths = getAuthorizations(conf);
+    byte[] password = getPassword(conf);
+    String username = getUsername(conf);
+    Instance instance = getInstance(conf);
+    List<AccumuloIterator> iterators = getIterators(conf);
+    List<AccumuloIteratorOption> options = getIteratorOptions(conf);
+    Level logLevel = getLogLevel(conf);
+
     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;
@@ -1314,7 +1421,6 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           binnedRanges = binOfflineTable(job, tableName, ranges);
         }
       } else {
-        Instance instance = getInstance(job.getConfiguration());
         String tableId = null;
         tl = getTabletLocator(job.getConfiguration());
         // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
@@ -1337,15 +1443,15 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     } catch (Exception e) {
       throw new IOException(e);
     }
-    
+
     ArrayList<InputSplit> splits = new ArrayList<InputSplit>(ranges.size());
     HashMap<Range,ArrayList<String>> splitsToAdd = null;
-    
+
     if (!autoAdjust)
       splitsToAdd = new HashMap<Range,ArrayList<String>>();
-    
+
     HashMap<String,String> hostNameCache = new HashMap<String,String>();
-    
+
     for (Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
       String ip = tserverBin.getKey().split(":", 2)[0];
       String location = hostNameCache.get(ip);
@@ -1354,14 +1460,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         location = inetAddress.getHostName();
         hostNameCache.put(ip, location);
       }
-      
+
       for (Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
         Range ke = extentRanges.getKey().toDataRange();
         for (Range r : extentRanges.getValue()) {
           if (autoAdjust) {
             // divide ranges into smaller ranges, based on the
             // tablets
-            splits.add(new RangeInputSplit(tableName, ke.clip(r), new String[] {location}));
+            splits.add(new RangeInputSplit(ke.clip(r), new String[] {location}));
           } else {
             // don't divide ranges
             ArrayList<String> locations = splitsToAdd.get(r);
@@ -1373,132 +1479,55 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         }
       }
     }
-    
+
     if (!autoAdjust)
       for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
-        splits.add(new RangeInputSplit(tableName, entry.getKey(), entry.getValue().toArray(new String[0])));
-    return splits;
-  }
-  
-  /**
-   * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
-   */
-  public static class RangeInputSplit extends InputSplit implements Writable {
-    private Range range;
-    private String[] locations;
-    
-    public RangeInputSplit() {
-      range = new Range();
-      locations = new String[0];
-    }
-    
-    public Range getRange() {
-      return range;
-    }
-    
-    private static byte[] extractBytes(ByteSequence seq, int numBytes) {
-      byte[] bytes = new byte[numBytes + 1];
-      bytes[0] = 0;
-      for (int i = 0; i < numBytes; i++) {
-        if (i >= seq.length())
-          bytes[i + 1] = 0;
-        else
-          bytes[i + 1] = seq.byteAt(i);
-      }
-      return bytes;
-    }
-    
-    public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
-      int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
-      BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
-      BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
-      BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
-      return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
-    }
-    
-    public float getProgress(Key currentKey) {
-      if (currentKey == null)
-        return 0f;
-      if (range.getStartKey() != null && range.getEndKey() != null) {
-        if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
-          // just look at the row progress
-          return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
-        } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
-          // just look at the column family progress
-          return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
-        } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL) != 0) {
-          // just look at the column qualifier progress
-          return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
-        }
-      }
-      // if we can't figure it out, then claim no progress
-      return 0f;
-    }
-    
-    RangeInputSplit(String table, Range range, String[] locations) {
-      this.range = range;
-      this.locations = locations;
-    }
-    
-    /**
-     * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
-     */
-    public long getLength() throws IOException {
-      Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
-      Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
-      int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
-      long diff = 0;
-      
-      byte[] start = startRow.getBytes();
-      byte[] stop = stopRow.getBytes();
-      for (int i = 0; i < maxCommon; ++i) {
-        diff |= 0xff & (start[i] ^ stop[i]);
-        diff <<= Byte.SIZE;
-      }
-      
-      if (startRow.getLength() != stopRow.getLength())
-        diff |= 0xff;
-      
-      return diff + 1;
-    }
-    
-    public String[] getLocations() throws IOException {
-      return locations;
-    }
-    
-    public void readFields(DataInput in) throws IOException {
-      range.readFields(in);
-      int numLocs = in.readInt();
-      locations = new String[numLocs];
-      for (int i = 0; i < numLocs; ++i)
-        locations[i] = in.readUTF();
-    }
-    
-    public void write(DataOutput out) throws IOException {
-      range.write(out);
-      out.writeInt(locations.length);
-      for (int i = 0; i < locations.length; ++i)
-        out.writeUTF(locations[i]);
+        splits.add(new RangeInputSplit(entry.getKey(), entry.getValue().toArray(new String[0])));
+
+    for (InputSplit inputSplit : splits) {
+      RangeInputSplit split = (RangeInputSplit) inputSplit;
+
+      split.setTable(tableName);
+      split.setOffline(offline);
+      split.setIsolatedScan(isolated);
+      split.setUsesLocalIterators(localIterators);
+      split.setMockInstance(mockInstance);
+      split.setMaxVersions(maxVersions);
+      split.setRowRegex(rowRegex);
+      split.setColfamRegex(colfamRegex);
+      split.setColqualRegex(colqualRegex);
+      split.setValueRegex(valueRegex);
+      split.setFetchedColumns(fetchedColumns);
+      split.setUsername(username);
+      split.setPassword(password);
+      split.setInstanceName(instance.getInstanceName());
+      split.setZooKeepers(instance.getZooKeepers());
+      split.setAuths(auths);
+      split.setIterators(iterators);
+      split.setOptions(options);
+      split.setLogLevel(logLevel);
     }
+
+    return splits;
   }
-  
+
   /**
    * The Class IteratorSetting. Encapsulates specifics for an Accumulo iterator's name & priority.
    */
   static class AccumuloIterator {
-    
+
     private static final String FIELD_SEP = ":";
-    
+
     private int priority;
     private String iteratorClass;
     private String iteratorName;
-    
+
     public AccumuloIterator(int priority, String iteratorClass, String iteratorName) {
       this.priority = priority;
       this.iteratorClass = iteratorClass;
       this.iteratorName = iteratorName;
     }
-    
+
     // Parses out a setting given an string supplied from an earlier toString() call
     public AccumuloIterator(String iteratorSetting) {
       // Parse the string to expand the iterator
@@ -1507,42 +1536,42 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       iteratorClass = tokenizer.nextToken();
       iteratorName = tokenizer.nextToken();
     }
-    
+
     public int getPriority() {
       return priority;
     }
-    
+
     public String getIteratorClass() {
       return iteratorClass;
     }
-    
+
     public String getIteratorName() {
       return iteratorName;
     }
-    
+
     @Override
     public String toString() {
       return new String(priority + FIELD_SEP + iteratorClass + FIELD_SEP + iteratorName);
     }
-    
+
   }
-  
+
   /**
    * The Class AccumuloIteratorOption. Encapsulates specifics for an Accumulo iterator's optional configuration details - associated via the iteratorName.
    */
   static class AccumuloIteratorOption {
     private static final String FIELD_SEP = ":";
-    
+
     private String iteratorName;
     private String key;
     private String value;
-    
+
     public AccumuloIteratorOption(String iteratorName, String key, String value) {
       this.iteratorName = iteratorName;
       this.key = key;
       this.value = value;
     }
-    
+
     // Parses out an option given a string supplied from an earlier toString() call
     public AccumuloIteratorOption(String iteratorOption) {
       StringTokenizer tokenizer = new StringTokenizer(iteratorOption, FIELD_SEP);
@@ -1554,19 +1583,19 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new RuntimeException(e);
       }
     }
-    
+
     public String getIteratorName() {
       return iteratorName;
     }
-    
+
     public String getKey() {
       return key;
     }
-    
+
     public String getValue() {
       return value;
     }
-    
+
     @Override
     public String toString() {
       try {
@@ -1575,7 +1604,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new RuntimeException(e);
       }
     }
-    
+
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/73114819/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
new file mode 100644
index 0000000..e372801
--- /dev/null
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -0,0 +1,493 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapreduce;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIterator;
+import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIteratorOption;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.PartialKey;
+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.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.Level;
+
+/**
+ * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
+ */
+public class RangeInputSplit extends InputSplit implements Writable {
+  private Range range;
+  private String[] locations;
+  private String table, instanceName, zooKeepers, username;
+  private String rowRegex, colfamRegex, colqualRegex, valueRegex;
+  private byte[] password;
+  private Boolean offline, mockInstance, isolatedScan, localIterators;
+  private Integer maxVersions;
+  private Authorizations auths;
+  private Set<Pair<Text,Text>> fetchedColumns;
+  private List<AccumuloIterator> iterators;
+  private List<AccumuloIteratorOption> options;
+  private Level level;
+
+  public RangeInputSplit() {
+    range = new Range();
+    locations = new String[0];
+  }
+
+  public RangeInputSplit(Range range, String[] locations) {
+    this.range = range;
+    this.locations = locations;
+  }
+
+  public Range getRange() {
+    return range;
+  }
+
+  private static byte[] extractBytes(ByteSequence seq, int numBytes) {
+    byte[] bytes = new byte[numBytes + 1];
+    bytes[0] = 0;
+    for (int i = 0; i < numBytes; i++) {
+      if (i >= seq.length())
+        bytes[i + 1] = 0;
+      else
+        bytes[i + 1] = seq.byteAt(i);
+    }
+    return bytes;
+  }
+
+  public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
+    int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
+    BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
+    BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
+    BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
+    return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
+  }
+
+  public float getProgress(Key currentKey) {
+    if (currentKey == null)
+      return 0f;
+    if (range.getStartKey() != null && range.getEndKey() != null) {
+      if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
+        // just look at the row progress
+        return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
+      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
+        // just look at the column family progress
+        return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
+      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL) != 0) {
+        // just look at the column qualifier progress
+        return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
+      }
+    }
+    // if we can't figure it out, then claim no progress
+    return 0f;
+  }
+
+  /**
+   * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
+   */
+  public long getLength() throws IOException {
+    Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
+    Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
+    int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
+    long diff = 0;
+
+    byte[] start = startRow.getBytes();
+    byte[] stop = stopRow.getBytes();
+    for (int i = 0; i < maxCommon; ++i) {
+      diff |= 0xff & (start[i] ^ stop[i]);
+      diff <<= Byte.SIZE;
+    }
+
+    if (startRow.getLength() != stopRow.getLength())
+      diff |= 0xff;
+
+    return diff + 1;
+  }
+
+  public String[] getLocations() throws IOException {
+    return locations;
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    range.readFields(in);
+    int numLocs = in.readInt();
+    locations = new String[numLocs];
+    for (int i = 0; i < numLocs; ++i)
+      locations[i] = in.readUTF();
+    
+    if (in.readBoolean()) {
+      isolatedScan = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      offline = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      localIterators = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      mockInstance = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      maxVersions = in.readInt();
+    }
+    
+    if (in.readBoolean()) {
+      rowRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      colfamRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      colqualRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      valueRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      int numColumns = in.readInt();
+      String[] columns = new String[numColumns];
+      for (int i = 0; i < numColumns; i++) {
+        columns[i] = in.readUTF();
+      }
+      
+      fetchedColumns = InputFormatBase.deserializeFetchedColumns(columns);
+    }
+    
+    if (in.readBoolean()) {
+      auths = new Authorizations(StringUtils.split(in.readUTF()));
+    }
+    
+    if (in.readBoolean()) {
+      username = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      password = in.readUTF().getBytes();
+    }
+    
+    if (in.readBoolean()) {
+      instanceName = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      zooKeepers = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      level = Level.toLevel(in.readInt());
+    }
+  }
+
+  public void write(DataOutput out) throws IOException {
+    range.write(out);
+    out.writeInt(locations.length);
+    for (int i = 0; i < locations.length; ++i)
+      out.writeUTF(locations[i]);
+    
+    out.writeBoolean(null != isolatedScan);
+    if (null != isolatedScan) {
+      out.writeBoolean(isolatedScan);
+    }
+    
+    out.writeBoolean(null != offline);
+    if (null != offline) {
+      out.writeBoolean(offline);
+    }
+    
+    out.writeBoolean(null != localIterators);
+    if (null != localIterators) {
+      out.writeBoolean(localIterators);
+    }
+    
+    out.writeBoolean(null != mockInstance);
+    if (null != mockInstance) {
+      out.writeBoolean(mockInstance);
+    }
+    
+    out.writeBoolean(null != maxVersions);
+    if (null != maxVersions) {
+      out.writeInt(getMaxVersions());
+    }
+    
+    out.writeBoolean(null != rowRegex);
+    if (null != rowRegex) {
+      out.writeUTF(rowRegex);
+    }
+    
+    out.writeBoolean(null != colfamRegex);
+    if (null != colfamRegex) {
+      out.writeUTF(colfamRegex);
+    }
+    
+    out.writeBoolean(null != colqualRegex);
+    if (null != colqualRegex) {
+      out.writeUTF(colqualRegex);
+    }
+    
+    out.writeBoolean(null != valueRegex);
+    if (null != valueRegex) {
+      out.writeUTF(valueRegex);
+    }
+    
+    out.writeBoolean(null != fetchedColumns);
+    if (null != fetchedColumns) {
+      String[] cols = InputFormatBase.serializeColumns(fetchedColumns);
+      out.writeInt(cols.length);
+      for (String col : cols) {
+        out.writeUTF(col);
+      }
+    }
+    
+    out.writeBoolean(null != auths);
+    if (null != auths) {
+      out.writeUTF(auths.serialize());
+    }
+    
+    out.writeBoolean(null != username);
+    if (null != username) {
+      out.writeUTF(username);
+    }
+    
+    out.writeBoolean(null != password);
+    if (null != password) {
+      out.writeUTF(new String(password));
+    }
+    
+    out.writeBoolean(null != instanceName);
+    if (null != instanceName) {
+      out.writeUTF(instanceName);
+    }
+    
+    out.writeBoolean(null != zooKeepers);
+    if (null != zooKeepers) {
+      out.writeUTF(zooKeepers);
+    }
+    
+    out.writeBoolean(null != level);
+    if (null != level) {
+      out.writeInt(level.toInt());
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder(128);
+    sb.append("Range: ").append(range);
+    sb.append(" Locations: ").append(locations);
+    sb.append(" Table: ").append(table);
+    // TODO finish building of string
+    return sb.toString();
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+  
+  public Instance getInstance() {
+    if (null == instanceName) {
+      return null;
+    }
+    
+    if (isMockInstance()) {  
+      return new MockInstance(getInstanceName());
+    }
+    
+    if (null == zooKeepers) {
+      return null;
+    }
+    
+    return new ZooKeeperInstance(getInstanceName(), getZooKeepers());
+  }
+
+  public String getInstanceName() {
+    return instanceName;
+  }
+
+  public void setInstanceName(String instanceName) {
+    this.instanceName = instanceName;
+  }
+
+  public String getZooKeepers() {
+    return zooKeepers;
+  }
+
+  public void setZooKeepers(String zooKeepers) {
+    this.zooKeepers = zooKeepers;
+  }
+
+  public String getUsername() {
+    return username;
+  }
+
+  public void setUsername(String username) {
+    this.username = username;
+  }
+
+  public byte[] getPassword() {
+    return password;
+  }
+
+  public void setPassword(byte[] password) {
+    this.password = password;
+  }
+
+  public Boolean isOffline() {
+    return offline;
+  }
+
+  public void setOffline(Boolean offline) {
+    this.offline = offline;
+  }
+
+  public void setLocations(String[] locations) {
+    this.locations = locations;
+  }
+
+  public String getRowRegex() {
+    return rowRegex;
+  }
+
+  public void setRowRegex(String rowRegex) {
+    this.rowRegex = rowRegex;
+  }
+
+  public String getColfamRegex() {
+    return colfamRegex;
+  }
+
+  public void setColfamRegex(String colfamRegex) {
+    this.colfamRegex = colfamRegex;
+  }
+
+  public String getColqualRegex() {
+    return colqualRegex;
+  }
+
+  public void setColqualRegex(String colqualRegex) {
+    this.colqualRegex = colqualRegex;
+  }
+
+  public String getValueRegex() {
+    return valueRegex;
+  }
+
+  public void setValueRegex(String valueRegex) {
+    this.valueRegex = valueRegex;
+  }
+
+  public Boolean isMockInstance() {
+    return mockInstance;
+  }
+
+  public void setMockInstance(Boolean mockInstance) {
+    this.mockInstance = mockInstance;
+  }
+
+  public Boolean isIsolatedScan() {
+    return isolatedScan;
+  }
+
+  public void setIsolatedScan(Boolean isolatedScan) {
+    this.isolatedScan = isolatedScan;
+  }
+
+  public Integer getMaxVersions() {
+    return maxVersions;
+  }
+
+  public void setMaxVersions(Integer maxVersions) {
+    this.maxVersions = maxVersions;
+  }
+
+  public Authorizations getAuths() {
+    return auths;
+  }
+
+  public void setAuths(Authorizations auths) {
+    this.auths = auths;
+  }
+
+  public void setRange(Range range) {
+    this.range = range;
+  }
+
+  public Boolean usesLocalIterators() {
+    return localIterators;
+  }
+
+  public void setUsesLocalIterators(Boolean localIterators) {
+    this.localIterators = localIterators;
+  }
+
+  public Set<Pair<Text,Text>> getFetchedColumns() {
+    return fetchedColumns;
+  }
+
+  public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
+    this.fetchedColumns = fetchedColumns;
+  }
+
+  public List<AccumuloIterator> getIterators() {
+    return iterators;
+  }
+
+  public void setIterators(List<AccumuloIterator> iterators) {
+    this.iterators = iterators;
+  }
+
+  public List<AccumuloIteratorOption> getOptions() {
+    return options;
+  }
+
+  public void setOptions(List<AccumuloIteratorOption> options) {
+    this.options = options;
+  }
+  
+  public Level getLogLevel() {
+    return level;
+  }
+  
+  public void setLogLevel(Level level) {
+    this.level = level;
+  }
+}


[13/19] git commit: Merge remote-tracking branch 'origin/1.5.1-SNAPSHOT' into ACCUMULO-1854-1.5-merge

Posted by el...@apache.org.
Merge remote-tracking branch 'origin/1.5.1-SNAPSHOT' into ACCUMULO-1854-1.5-merge


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/4f098247
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/4f098247
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/4f098247

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 4f0982473451815264da8c8d919424787fbf3122
Parents: 03b065d 7eaedc4
Author: Josh Elser <el...@apache.org>
Authored: Fri Nov 22 13:00:47 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Nov 22 13:00:47 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/server/util/MetadataTable.java  | 12 +++++++-----
 1 file changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[02/19] git commit: Squashed commit of the following:

Posted by el...@apache.org.
Squashed commit of the following:

commit dfbe098fb650d1d1605ac28ff0b195e229ecb345
Author: Josh Elser <el...@apache.org>
Date:   Wed Nov 20 23:57:18 2013 -0500

    ACCUMULO-1843 Add in log4j Level to RangeInputSplit. Add more tests, notably ones that exercise delegation of the input
    split to the Configuration.

commit 38fdee9916edd938bea1642de5d4e5cf54a81596
Author: Josh Elser <el...@apache.org>
Date:   Fri Nov 8 17:47:57 2013 -0500

    ACCUMULO-1854 Fix up InputFormatBase to use the information stored on
    RangeInputSplit and fall back onto the Configuration.

commit 0e6d1aba7eacef357e0a17c67a453dd5b50a49dc
Author: Josh Elser <el...@apache.org>
Date:   Fri Nov 8 16:23:49 2013 -0500

    ACCUMULO-1854 Clean up constructors. Add a test.

commit 2f59f81f6e75f8a90ccfe3df00c6ad3f69174e0c
Author: Josh Elser <el...@apache.org>
Date:   Fri Nov 8 15:46:39 2013 -0500

    ACCUMULO-1854 Move RangeInputSplit into its own file and store all
    connection information into it.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/45ae55fc
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/45ae55fc
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/45ae55fc

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 45ae55fcb74832983ffc188524790a56e5261ae0
Parents: 3f633bf
Author: Josh Elser <el...@apache.org>
Authored: Thu Nov 21 00:19:59 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Nov 21 00:19:59 2013 -0500

----------------------------------------------------------------------
 .../client/mapreduce/AccumuloInputFormat.java   |  11 +
 .../core/client/mapreduce/InputFormatBase.java  | 615 ++++++++++---------
 .../core/client/mapreduce/RangeInputSplit.java  | 493 +++++++++++++++
 .../mapreduce/AccumuloInputFormatTest.java      | 253 ++++++--
 .../mapreduce/AccumuloRowInputFormatTest.java   |   1 -
 .../client/mapreduce/RangeInputSplitTest.java   | 100 +++
 .../simple/filedata/ChunkInputFormatTest.java   |   4 +-
 7 files changed, 1126 insertions(+), 351 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/45ae55fc/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
index 4de131f..c9a70eb 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.util.format.DefaultFormatter;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.log4j.Level;
 
 /**
  * This class allows MapReduce jobs to use Accumulo as the source of data. This input format provides keys and values of type Key and Value to the Map() and
@@ -44,6 +45,16 @@ public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
   @Override
   public RecordReader<Key,Value> createRecordReader(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
     log.setLevel(getLogLevel(context.getConfiguration()));
+    
+    // Override the log level from the configuration as if the RangeInputSplit has one it's the more correct one to use.
+    if (split instanceof RangeInputSplit) {
+      RangeInputSplit risplit = (RangeInputSplit) split;
+      Level level = risplit.getLogLevel();
+      if (null != level) {
+        log.setLevel(level);
+      }
+    }
+
     return new RecordReaderBase<Key,Value>() {
       @Override
       public boolean nextKeyValue() throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/45ae55fc/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index 8e238f1..40e09a1 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -110,7 +110,7 @@ import org.apache.log4j.Logger;
 
 public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static final Logger log = Logger.getLogger(InputFormatBase.class);
-  
+
   private static final String PREFIX = AccumuloInputFormat.class.getSimpleName();
   private static final String INPUT_INFO_HAS_BEEN_SET = PREFIX + ".configured";
   private static final String INSTANCE_HAS_BEEN_SET = PREFIX + ".instanceConfigured";
@@ -118,34 +118,34 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   private static final String PASSWORD = PREFIX + ".password";
   private static final String TABLE_NAME = PREFIX + ".tablename";
   private static final String AUTHORIZATIONS = PREFIX + ".authorizations";
-  
+
   private static final String INSTANCE_NAME = PREFIX + ".instanceName";
   private static final String ZOOKEEPERS = PREFIX + ".zooKeepers";
   private static final String MOCK = ".useMockInstance";
-  
+
   private static final String RANGES = PREFIX + ".ranges";
   private static final String AUTO_ADJUST_RANGES = PREFIX + ".ranges.autoAdjust";
-  
+
   private static final String ROW_REGEX = PREFIX + ".regex.row";
   private static final String COLUMN_FAMILY_REGEX = PREFIX + ".regex.cf";
   private static final String COLUMN_QUALIFIER_REGEX = PREFIX + ".regex.cq";
   private static final String VALUE_REGEX = PREFIX + ".regex.value";
-  
+
   private static final String COLUMNS = PREFIX + ".columns";
   private static final String LOGLEVEL = PREFIX + ".loglevel";
-  
+
   private static final String ISOLATED = PREFIX + ".isolated";
-  
+
   private static final String LOCAL_ITERATORS = PREFIX + ".localiters";
-  
+
   // Used to specify the maximum # of versions of an Accumulo cell value to return
   private static final String MAX_VERSIONS = PREFIX + ".maxVersions";
-  
+
   // Used for specifying the iterators to be applied
   private static final String ITERATORS = PREFIX + ".iterators";
   private static final String ITERATORS_OPTIONS = PREFIX + ".iterators.options";
   private static final String ITERATORS_DELIM = ",";
-  
+
   private static final String READ_OFFLINE = PREFIX + ".read.offline";
 
   /**
@@ -154,7 +154,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIsolated(JobContext job, boolean enable) {
     setIsolated(job.getConfiguration(), enable);
   }
-  
+
   /**
    * Enable or disable use of the {@link IsolatedScanner} in this configuration object. By default it is not enabled.
    * 
@@ -166,14 +166,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIsolated(Configuration conf, boolean enable) {
     conf.setBoolean(ISOLATED, enable);
   }
-  
+
   /**
    * @deprecated Use {@link #setLocalIterators(Configuration,boolean)} instead
    */
   public static void setLocalIterators(JobContext job, boolean enable) {
     setLocalIterators(job.getConfiguration(), enable);
   }
-  
+
   /**
    * Enable or disable use of the {@link ClientSideIteratorScanner} in this Configuration object. By default it is not enabled.
    * 
@@ -185,14 +185,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setLocalIterators(Configuration conf, boolean enable) {
     conf.setBoolean(LOCAL_ITERATORS, enable);
   }
-  
+
   /**
    * @deprecated Use {@link #setInputInfo(Configuration,String,byte[],String,Authorizations)} instead
    */
   public static void setInputInfo(JobContext job, String user, byte[] passwd, String table, Authorizations auths) {
     setInputInfo(job.getConfiguration(), user, passwd, table, auths);
   }
-  
+
   /**
    * Initialize the user, table, and authorization information for the configuration object that will be used with an Accumulo InputFormat.
    * 
@@ -211,7 +211,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     if (conf.getBoolean(INPUT_INFO_HAS_BEEN_SET, false))
       throw new IllegalStateException("Input info can only be set once per job");
     conf.setBoolean(INPUT_INFO_HAS_BEEN_SET, true);
-    
+
     ArgumentChecker.notNull(user, passwd, table);
     conf.set(USERNAME, user);
     conf.set(PASSWORD, new String(Base64.encodeBase64(passwd)));
@@ -219,14 +219,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     if (auths != null && !auths.isEmpty())
       conf.set(AUTHORIZATIONS, auths.serialize());
   }
-  
+
   /**
    * @deprecated Use {@link #setZooKeeperInstance(Configuration,String,String)} instead
    */
   public static void setZooKeeperInstance(JobContext job, String instanceName, String zooKeepers) {
     setZooKeeperInstance(job.getConfiguration(), instanceName, zooKeepers);
   }
-  
+
   /**
    * Configure a {@link ZooKeeperInstance} for this configuration object.
    * 
@@ -241,19 +241,19 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     if (conf.getBoolean(INSTANCE_HAS_BEEN_SET, false))
       throw new IllegalStateException("Instance info can only be set once per job");
     conf.setBoolean(INSTANCE_HAS_BEEN_SET, true);
-    
+
     ArgumentChecker.notNull(instanceName, zooKeepers);
     conf.set(INSTANCE_NAME, instanceName);
     conf.set(ZOOKEEPERS, zooKeepers);
   }
-  
+
   /**
    * @deprecated Use {@link #setMockInstance(Configuration,String)} instead
    */
   public static void setMockInstance(JobContext job, String instanceName) {
     setMockInstance(job.getConfiguration(), instanceName);
   }
-  
+
   /**
    * Configure a {@link MockInstance} for this configuration object.
    * 
@@ -267,14 +267,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     conf.setBoolean(MOCK, true);
     conf.set(INSTANCE_NAME, instanceName);
   }
-  
+
   /**
    * @deprecated Use {@link #setRanges(Configuration,Collection)} instead
    */
   public static void setRanges(JobContext job, Collection<Range> ranges) {
     setRanges(job.getConfiguration(), ranges);
   }
-  
+
   /**
    * Set the ranges to map over for this configuration object.
    * 
@@ -297,14 +297,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     conf.setStrings(RANGES, rangeStrings.toArray(new String[0]));
   }
-  
+
   /**
    * @deprecated Use {@link #disableAutoAdjustRanges(Configuration)} instead
    */
   public static void disableAutoAdjustRanges(JobContext job) {
     disableAutoAdjustRanges(job.getConfiguration());
   }
-  
+
   /**
    * Disables the adjustment of ranges for this configuration object. By default, overlapping ranges will be merged and ranges will be fit to existing tablet
    * boundaries. Disabling this adjustment will cause there to be exactly one mapper per range set using {@link #setRanges(Configuration, Collection)}.
@@ -315,14 +315,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void disableAutoAdjustRanges(Configuration conf) {
     conf.setBoolean(AUTO_ADJUST_RANGES, false);
   }
-  
+
   /**
    * @deprecated since 1.4 use {@link org.apache.accumulo.core.iterators.user.RegExFilter} and {@link #addIterator(Configuration, IteratorSetting)}
    */
   public static enum RegexType {
     ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VALUE
   }
-  
+
   /**
    * @deprecated since 1.4 use {@link #addIterator(Configuration, IteratorSetting)}
    * @see org.apache.accumulo.core.iterators.user.RegExFilter#setRegexs(IteratorSetting, String, String, String, String, boolean)
@@ -356,14 +356,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * @deprecated Use {@link #setMaxVersions(Configuration,int)} instead
    */
   public static void setMaxVersions(JobContext job, int maxVersions) throws IOException {
     setMaxVersions(job.getConfiguration(), maxVersions);
   }
-  
+
   /**
    * Sets the max # of values that may be returned for an individual Accumulo cell. By default, applied before all other Accumulo iterators (highest priority)
    * leveraged in the scan by the record reader. To adjust priority use setIterator() & setIteratorOptions() w/ the VersioningIterator type explicitly.
@@ -380,7 +380,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       throw new IOException("Invalid maxVersions: " + maxVersions + ".  Must be >= 1");
     conf.setInt(MAX_VERSIONS, maxVersions);
   }
-  
+
   /**
    * <p>
    * Enable reading offline tables. This will make the map reduce job directly read the tables files. If the table is not offline, then the job will fail. If
@@ -407,18 +407,18 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @param scanOff
    *          pass true to read offline tables
    */
-  
+
   public static void setScanOffline(Configuration conf, boolean scanOff) {
     conf.setBoolean(READ_OFFLINE, scanOff);
   }
-  
+
   /**
    * @deprecated Use {@link #fetchColumns(Configuration,Collection)} instead
    */
   public static void fetchColumns(JobContext job, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
     fetchColumns(job.getConfiguration(), columnFamilyColumnQualifierPairs);
   }
-  
+
   /**
    * Restricts the columns that will be mapped over for this configuration object.
    * 
@@ -429,27 +429,33 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    *          selected. An empty set is the default and is equivalent to scanning the all columns.
    */
   public static void fetchColumns(Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
+    String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
+    conf.setStrings(COLUMNS, columnStrings);
+  }
+
+  public static String[] serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
     ArgumentChecker.notNull(columnFamilyColumnQualifierPairs);
     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())));
       if (column.getSecond() != null)
         col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())));
       columnStrings.add(col);
     }
-    conf.setStrings(COLUMNS, columnStrings.toArray(new String[0]));
+
+    return columnStrings.toArray(new String[0]);
   }
-  
+
   /**
    * @deprecated Use {@link #setLogLevel(Configuration,Level)} instead
    */
   public static void setLogLevel(JobContext job, Level level) {
     setLogLevel(job.getConfiguration(), level);
   }
-  
+
   /**
    * Sets the log level for this configuration object.
    * 
@@ -463,14 +469,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     log.setLevel(level);
     conf.setInt(LOGLEVEL, level.toInt());
   }
-  
+
   /**
    * @deprecated Use {@link #addIterator(Configuration,IteratorSetting)} instead
    */
   public static void addIterator(JobContext job, IteratorSetting cfg) {
     addIterator(job.getConfiguration(), cfg);
   }
-  
+
   /**
    * Encode an iterator on the input for this configuration object.
    * 
@@ -482,7 +488,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void addIterator(Configuration conf, IteratorSetting cfg) {
     // First check to see if anything has been set already
     String iterators = conf.get(ITERATORS);
-    
+
     // No iterators specified yet, create a new string
     if (iterators == null || iterators.isEmpty()) {
       iterators = new AccumuloIterator(cfg.getPriority(), cfg.getIteratorClass(), cfg.getName()).toString();
@@ -495,9 +501,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     for (Entry<String,String> entry : cfg.getOptions().entrySet()) {
       if (entry.getValue() == null)
         continue;
-      
+
       String iteratorOptions = conf.get(ITERATORS_OPTIONS);
-      
+
       // No options specified yet, create a new string
       if (iteratorOptions == null || iteratorOptions.isEmpty()) {
         iteratorOptions = new AccumuloIteratorOption(cfg.getName(), entry.getKey(), entry.getValue()).toString();
@@ -505,12 +511,12 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         // append the next option & reset
         iteratorOptions = iteratorOptions.concat(ITERATORS_DELIM + new AccumuloIteratorOption(cfg.getName(), entry.getKey(), entry.getValue()));
       }
-      
+
       // Store the options w/ the job
       conf.set(ITERATORS_OPTIONS, iteratorOptions);
     }
   }
-  
+
   /**
    * Specify an Accumulo iterator type to manage the behavior of the underlying table scan this InputFormat's RecordReader will conduct, w/ priority dictating
    * the order in which specified iterators are applied. Repeat calls to specify multiple iterators are allowed.
@@ -529,7 +535,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIterator(JobContext job, int priority, String iteratorClass, String iteratorName) {
     // First check to see if anything has been set already
     String iterators = job.getConfiguration().get(ITERATORS);
-    
+
     // No iterators specified yet, create a new string
     if (iterators == null || iterators.isEmpty()) {
       iterators = new AccumuloIterator(priority, iteratorClass, iteratorName).toString();
@@ -539,9 +545,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     // Store the iterators w/ the job
     job.getConfiguration().set(ITERATORS, iterators);
-    
+
   }
-  
+
   /**
    * Specify an option for a named Accumulo iterator, further specifying that iterator's behavior.
    * 
@@ -559,9 +565,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setIteratorOption(JobContext job, String iteratorName, String key, String value) {
     if (iteratorName == null || key == null || value == null)
       return;
-    
+
     String iteratorOptions = job.getConfiguration().get(ITERATORS_OPTIONS);
-    
+
     // No options specified yet, create a new string
     if (iteratorOptions == null || iteratorOptions.isEmpty()) {
       iteratorOptions = new AccumuloIteratorOption(iteratorName, key, value).toString();
@@ -569,18 +575,18 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       // append the next option & reset
       iteratorOptions = iteratorOptions.concat(ITERATORS_DELIM + new AccumuloIteratorOption(iteratorName, key, value));
     }
-    
+
     // Store the options w/ the job
     job.getConfiguration().set(ITERATORS_OPTIONS, iteratorOptions);
   }
-  
+
   /**
    * @deprecated Use {@link #isIsolated(Configuration)} instead
    */
   protected static boolean isIsolated(JobContext job) {
     return isIsolated(job.getConfiguration());
   }
-  
+
   /**
    * Determines whether a configuration has isolation enabled.
    * 
@@ -592,14 +598,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean isIsolated(Configuration conf) {
     return conf.getBoolean(ISOLATED, false);
   }
-  
+
   /**
    * @deprecated Use {@link #usesLocalIterators(Configuration)} instead
    */
   protected static boolean usesLocalIterators(JobContext job) {
     return usesLocalIterators(job.getConfiguration());
   }
-  
+
   /**
    * Determines whether a configuration uses local iterators.
    * 
@@ -611,14 +617,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean usesLocalIterators(Configuration conf) {
     return conf.getBoolean(LOCAL_ITERATORS, false);
   }
-  
+
   /**
    * @deprecated Use {@link #getUsername(Configuration)} instead
    */
   protected static String getUsername(JobContext job) {
     return getUsername(job.getConfiguration());
   }
-  
+
   /**
    * Gets the user name from the configuration.
    * 
@@ -630,7 +636,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static String getUsername(Configuration conf) {
     return conf.get(USERNAME);
   }
-  
+
   /**
    * WARNING: The password 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.
@@ -640,7 +646,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static byte[] getPassword(JobContext job) {
     return getPassword(job.getConfiguration());
   }
-  
+
   /**
    * Gets the password from the configuration. WARNING: The password 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.
@@ -653,14 +659,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static byte[] getPassword(Configuration conf) {
     return Base64.decodeBase64(conf.get(PASSWORD, "").getBytes());
   }
-  
+
   /**
    * @deprecated Use {@link #getTablename(Configuration)} instead
    */
   protected static String getTablename(JobContext job) {
     return getTablename(job.getConfiguration());
   }
-  
+
   /**
    * Gets the table name from the configuration.
    * 
@@ -672,14 +678,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static String getTablename(Configuration conf) {
     return conf.get(TABLE_NAME);
   }
-  
+
   /**
    * @deprecated Use {@link #getAuthorizations(Configuration)} instead
    */
   protected static Authorizations getAuthorizations(JobContext job) {
     return getAuthorizations(job.getConfiguration());
   }
-  
+
   /**
    * Gets the authorizations to set for the scans from the configuration.
    * 
@@ -692,14 +698,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     String authString = conf.get(AUTHORIZATIONS);
     return authString == null ? Constants.NO_AUTHS : new Authorizations(authString.split(","));
   }
-  
+
   /**
    * @deprecated Use {@link #getInstance(Configuration)} instead
    */
   protected static Instance getInstance(JobContext job) {
     return getInstance(job.getConfiguration());
   }
-  
+
   /**
    * Initializes an Accumulo {@link Instance} based on the configuration.
    * 
@@ -714,14 +720,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       return new MockInstance(conf.get(INSTANCE_NAME));
     return new ZooKeeperInstance(conf.get(INSTANCE_NAME), conf.get(ZOOKEEPERS));
   }
-  
+
   /**
    * @deprecated Use {@link #getTabletLocator(Configuration)} instead
    */
   protected static TabletLocator getTabletLocator(JobContext job) throws TableNotFoundException {
     return getTabletLocator(job.getConfiguration());
   }
-  
+
   /**
    * Initializes an Accumulo {@link TabletLocator} based on the configuration.
    * 
@@ -741,14 +747,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     return TabletLocator.getInstance(instance, new AuthInfo(username, ByteBuffer.wrap(password), instance.getInstanceID()),
         new Text(Tables.getTableId(instance, tableName)));
   }
-  
+
   /**
    * @deprecated Use {@link #getRanges(Configuration)} instead
    */
   protected static List<Range> getRanges(JobContext job) throws IOException {
     return getRanges(job.getConfiguration());
   }
-  
+
   /**
    * Gets the ranges to scan over from a configuration object.
    * 
@@ -769,7 +775,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return ranges;
   }
-  
+
   /**
    * @deprecated since 1.4 use {@link org.apache.accumulo.core.iterators.user.RegExFilter} and {@link #addIterator(Configuration, IteratorSetting)}
    * @see #setRegex(JobContext, RegexType, String)
@@ -802,14 +808,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * @deprecated Use {@link #getFetchedColumns(Configuration)} instead
    */
   protected static Set<Pair<Text,Text>> getFetchedColumns(JobContext job) {
     return getFetchedColumns(job.getConfiguration());
   }
-  
+
   /**
    * Gets the columns to be mapped over from this configuration object.
    * 
@@ -819,8 +825,19 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @see #fetchColumns(Configuration, Collection)
    */
   protected static Set<Pair<Text,Text>> getFetchedColumns(Configuration conf) {
+    ArgumentChecker.notNull(conf);
+
+    return deserializeFetchedColumns(conf.getStrings(COLUMNS));
+  }
+
+  public static Set<Pair<Text,Text>> deserializeFetchedColumns(String[] serialized) {
     Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
-    for (String col : conf.getStringCollection(COLUMNS)) {
+
+    if (null == serialized) {
+      return columns;
+    }
+
+    for (String col : serialized) {
       int idx = col.indexOf(":");
       Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes()) : Base64.decodeBase64(col.substring(0, idx).getBytes()));
       Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes()));
@@ -828,14 +845,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return columns;
   }
-  
+
   /**
    * @deprecated Use {@link #getAutoAdjustRanges(Configuration)} instead
    */
   protected static boolean getAutoAdjustRanges(JobContext job) {
     return getAutoAdjustRanges(job.getConfiguration());
   }
-  
+
   /**
    * Determines whether a configuration has auto-adjust ranges enabled.
    * 
@@ -847,14 +864,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean getAutoAdjustRanges(Configuration conf) {
     return conf.getBoolean(AUTO_ADJUST_RANGES, true);
   }
-  
+
   /**
    * @deprecated Use {@link #getLogLevel(Configuration)} instead
    */
   protected static Level getLogLevel(JobContext job) {
     return getLogLevel(job.getConfiguration());
   }
-  
+
   /**
    * Gets the log level from this configuration.
    * 
@@ -866,7 +883,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static Level getLogLevel(Configuration conf) {
     return Level.toLevel(conf.getInt(LOGLEVEL, Level.INFO.toInt()));
   }
-  
+
   // InputFormat doesn't have the equivalent of OutputFormat's
   // checkOutputSpecs(JobContext job)
   /**
@@ -875,7 +892,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static void validateOptions(JobContext job) throws IOException {
     validateOptions(job.getConfiguration());
   }
-  
+
   // InputFormat doesn't have the equivalent of OutputFormat's
   // checkOutputSpecs(JobContext job)
   /**
@@ -898,7 +915,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new IOException("Unable to authenticate user");
       if (!c.securityOperations().hasTablePermission(getUsername(conf), getTablename(conf), TablePermission.READ))
         throw new IOException("Unable to access table");
-      
+
       if (!usesLocalIterators(conf)) {
         // validate that any scan-time iterators can be loaded by the the tablet servers
         for (AccumuloIterator iter : getIterators(conf)) {
@@ -906,21 +923,21 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
             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);
     }
   }
-  
+
   /**
    * @deprecated Use {@link #getMaxVersions(Configuration)} instead
    */
   protected static int getMaxVersions(JobContext job) {
     return getMaxVersions(job.getConfiguration());
   }
-  
+
   /**
    * Gets the maxVersions to use for the {@link VersioningIterator} from this configuration.
    * 
@@ -932,7 +949,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static int getMaxVersions(Configuration conf) {
     return conf.getInt(MAX_VERSIONS, -1);
   }
-  
+
   protected static boolean isOfflineScan(Configuration conf) {
     return conf.getBoolean(READ_OFFLINE, false);
   }
@@ -945,7 +962,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static List<AccumuloIterator> getIterators(JobContext job) {
     return getIterators(job.getConfiguration());
   }
-  
+
   /**
    * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
    * 
@@ -955,13 +972,13 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @see #addIterator(Configuration, IteratorSetting)
    */
   protected static List<AccumuloIterator> getIterators(Configuration conf) {
-    
+
     String iterators = conf.get(ITERATORS);
-    
+
     // If no iterators are present, return an empty list
     if (iterators == null || iterators.isEmpty())
       return new ArrayList<AccumuloIterator>();
-    
+
     // Compose the set of iterators encoded in the job configuration
     StringTokenizer tokens = new StringTokenizer(conf.get(ITERATORS), ITERATORS_DELIM);
     List<AccumuloIterator> list = new ArrayList<AccumuloIterator>();
@@ -971,14 +988,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return list;
   }
-  
+
   /**
    * @deprecated Use {@link #getIteratorOptions(Configuration)} instead
    */
   protected static List<AccumuloIteratorOption> getIteratorOptions(JobContext job) {
     return getIteratorOptions(job.getConfiguration());
   }
-  
+
   /**
    * Gets a list of the iterator options specified on this configuration.
    * 
@@ -989,11 +1006,11 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    */
   protected static List<AccumuloIteratorOption> getIteratorOptions(Configuration conf) {
     String iteratorOptions = conf.get(ITERATORS_OPTIONS);
-    
+
     // If no options are present, return an empty list
     if (iteratorOptions == null || iteratorOptions.isEmpty())
       return new ArrayList<AccumuloIteratorOption>();
-    
+
     // Compose the set of options encoded in the job configuration
     StringTokenizer tokens = new StringTokenizer(conf.get(ITERATORS_OPTIONS), ITERATORS_DELIM);
     List<AccumuloIteratorOption> list = new ArrayList<AccumuloIteratorOption>();
@@ -1003,13 +1020,13 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
     return list;
   }
-  
+
   protected abstract static class RecordReaderBase<K,V> extends RecordReader<K,V> {
     protected long numKeysRead;
     protected Iterator<Entry<Key,Value>> scannerIterator;
     private boolean scannerRegexEnabled = false;
     protected RangeInputSplit split;
-    
+
     /**
      * @deprecated since 1.4, configure {@link org.apache.accumulo.core.iterators.user.RegExFilter} instead.
      */
@@ -1024,7 +1041,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         log.info("Setting " + methodName + " to " + regex);
       }
     }
-    
+
     /**
      * @deprecated since 1.4, configure {@link org.apache.accumulo.core.iterators.user.RegExFilter} instead.
      */
@@ -1039,15 +1056,16 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new AccumuloException("Can't set up regex for scanner");
       }
     }
-    
+
     // Apply the configured iterators from the job to the scanner
     /**
      * @deprecated Use {@link #setupIterators(Configuration,Scanner)} instead
      */
-    protected void setupIterators(TaskAttemptContext attempt, Scanner scanner) throws AccumuloException {
-      setupIterators(attempt.getConfiguration(), scanner);
+    protected void setupIterators(TaskAttemptContext attempt, Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
+        throws AccumuloException {
+      setupIterators(attempt.getConfiguration(), scanner, iterators, options);
     }
-    
+
     /**
      * Apply the configured iterators from the configuration to the scanner.
      * 
@@ -1057,10 +1075,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
      *          the scanner to configure
      * @throws AccumuloException
      */
-    protected void setupIterators(Configuration conf, Scanner scanner) throws AccumuloException {
-      List<AccumuloIterator> iterators = getIterators(conf);
-      List<AccumuloIteratorOption> options = getIteratorOptions(conf);
-      
+    protected void setupIterators(Configuration conf, Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
+        throws AccumuloException {
+
       Map<String,IteratorSetting> scanIterators = new HashMap<String,IteratorSetting>();
       for (AccumuloIterator iterator : iterators) {
         scanIterators.put(iterator.getIteratorName(), new IteratorSetting(iterator.getPriority(), iterator.getIteratorName(), iterator.getIteratorClass()));
@@ -1072,14 +1089,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         scanner.addScanIterator(scanIterators.get(iterator.getIteratorName()));
       }
     }
-    
+
     /**
      * @deprecated Use {@link #setupMaxVersions(Configuration,Scanner)} instead
      */
-    protected void setupMaxVersions(TaskAttemptContext attempt, Scanner scanner) {
-      setupMaxVersions(attempt.getConfiguration(), scanner);
+    protected void setupMaxVersions(TaskAttemptContext attempt, Scanner scanner, int maxVersions) {
+      setupMaxVersions(attempt.getConfiguration(), scanner, maxVersions);
     }
-    
+
     /**
      * If maxVersions has been set, configure a {@link VersioningIterator} at priority 0 for this scanner.
      * 
@@ -1088,8 +1105,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
      * @param scanner
      *          the scanner to configure
      */
-    protected void setupMaxVersions(Configuration conf, Scanner scanner) {
-      int maxVersions = getMaxVersions(conf);
+    protected void setupMaxVersions(Configuration conf, Scanner scanner, int maxVersions) {
       // Check to make sure its a legit value
       if (maxVersions >= 1) {
         IteratorSetting vers = new IteratorSetting(0, "vers", VersioningIterator.class);
@@ -1097,54 +1113,128 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         scanner.addScanIterator(vers);
       }
     }
-    
+
     /**
      * Initialize a scanner over the given input split using this task attempt configuration.
      */
     public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
       Scanner scanner;
       split = (RangeInputSplit) inSplit;
-      log.debug("Initializing input split: " + split.range);
+      log.debug("Initializing input split: " + split.getRange());
       Configuration conf = attempt.getConfiguration();
-      Instance instance = getInstance(conf);
-      String user = getUsername(conf);
-      byte[] password = getPassword(conf);
-      Authorizations authorizations = getAuthorizations(conf);
+
+      Instance instance = split.getInstance();
+      if (null == instance) {
+        instance = getInstance(conf);
+      }
+
+      String user = split.getUsername();
+      if (null == user) {
+        user = getUsername(conf);
+      }
+
+      byte[] password = split.getPassword();
+      if (null == password) {
+        password = getPassword(conf);
+      }
+
+      Authorizations authorizations = split.getAuths();
+      if (null == authorizations) {
+        authorizations = getAuthorizations(conf);
+      }
+
+      String table = split.getTable();
+      if (null == table) {
+        table = getTablename(conf);
+      }
+      
+      Boolean isOffline = split.isOffline();
+      if (null == isOffline) {
+        isOffline = isOfflineScan(conf);
+      }
+
+      Boolean isIsolated = split.isIsolatedScan();
+      if (null == isIsolated) {
+        isIsolated = isIsolated(conf);
+      }
+
+      Boolean usesLocalIterators = split.usesLocalIterators();
+      if (null == usesLocalIterators) {
+        usesLocalIterators = usesLocalIterators(conf);
+      }
+
+      String rowRegex = split.getRowRegex();
+      if (null == rowRegex) {
+        rowRegex = conf.get(ROW_REGEX);
+      }
+
+      String colfRegex = split.getColfamRegex();
+      if (null == colfRegex) {
+        colfRegex = conf.get(COLUMN_FAMILY_REGEX);
+      }
+
+      String colqRegex = split.getColqualRegex();
+      if (null == colqRegex) {
+        colqRegex = conf.get(COLUMN_QUALIFIER_REGEX);
+      }
+
+      String valueRegex = split.getValueRegex();
+      if (null == valueRegex) {
+        valueRegex = conf.get(VALUE_REGEX);
+      }
+
+      Integer maxVersions = split.getMaxVersions();
+      if (null == maxVersions) {
+        maxVersions = getMaxVersions(conf);
+      }
+      
+      List<AccumuloIterator> iterators = split.getIterators();
+      if (null == iterators) {
+        iterators = getIterators(conf);
+      }
+      
+      List<AccumuloIteratorOption> options = split.getOptions();
+      if (null == options) {
+        options = getIteratorOptions(conf);
+      }
       
+      Set<Pair<Text,Text>> columns = split.getFetchedColumns();
+      if (null == columns) {
+        columns = getFetchedColumns(conf);
+      }
+
       try {
         log.debug("Creating connector with user: " + user);
         Connector conn = instance.getConnector(user, password);
-        log.debug("Creating scanner for table: " + getTablename(conf));
+        log.debug("Creating scanner for table: " + table);
         log.debug("Authorizations are: " + authorizations);
-        if (isOfflineScan(conf)) {
-          scanner = new OfflineScanner(instance, new AuthInfo(user, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(instance,
-              getTablename(conf)), authorizations);
+        if (isOffline) {
+          scanner = new OfflineScanner(instance, new AuthInfo(user, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(instance, table),
+              authorizations);
         } else {
-          scanner = conn.createScanner(getTablename(conf), authorizations);
+          scanner = conn.createScanner(table, authorizations);
         }
-        if (isIsolated(conf)) {
+        if (isIsolated) {
           log.info("Creating isolated scanner");
           scanner = new IsolatedScanner(scanner);
         }
-        if (usesLocalIterators(conf)) {
+        if (usesLocalIterators) {
           log.info("Using local iterators");
           scanner = new ClientSideIteratorScanner(scanner);
         }
-        setupMaxVersions(conf, scanner);
-        if (conf.get(ROW_REGEX) != null || conf.get(COLUMN_FAMILY_REGEX) != null || conf.get(COLUMN_QUALIFIER_REGEX) != null ||
-            conf.get(VALUE_REGEX) != null) {
+        setupMaxVersions(conf, scanner, maxVersions);
+        if (rowRegex != null || colfRegex != null || colqRegex != null || valueRegex != null) {
           IteratorSetting is = new IteratorSetting(50, RegExFilter.class);
-          RegExFilter.setRegexs(is, conf.get(ROW_REGEX), conf.get(COLUMN_FAMILY_REGEX), conf.get(COLUMN_QUALIFIER_REGEX),
-            conf.get(VALUE_REGEX), false);
+          RegExFilter.setRegexs(is, rowRegex, colfRegex, colqRegex, valueRegex, false);
           scanner.addScanIterator(is);
         }
-        setupIterators(conf, scanner);
+        setupIterators(conf, scanner, iterators, options);
       } catch (Exception e) {
         throw new IOException(e);
       }
-      
+
       // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : getFetchedColumns(conf)) {
+      for (Pair<Text,Text> c : columns) {
         if (c.getSecond() != null) {
           log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
           scanner.fetchColumn(c.getFirst(), c.getSecond());
@@ -1153,48 +1243,48 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           scanner.fetchColumnFamily(c.getFirst());
         }
       }
-      
-      scanner.setRange(split.range);
-      
+
+      scanner.setRange(split.getRange());
+
       numKeysRead = 0;
-      
+
       // do this last after setting all scanner options
       scannerIterator = scanner.iterator();
     }
-    
+
     public void close() {}
-    
+
     public float getProgress() throws IOException {
       if (numKeysRead > 0 && currentKey == null)
         return 1.0f;
       return split.getProgress(currentKey);
     }
-    
+
     protected K currentK = null;
     protected V currentV = null;
     protected Key currentKey = null;
     protected Value currentValue = null;
-    
+
     @Override
     public K getCurrentKey() throws IOException, InterruptedException {
       return currentK;
     }
-    
+
     @Override
     public V getCurrentValue() throws IOException, InterruptedException {
       return currentV;
     }
   }
-  
+
   Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext job, String tableName, List<Range> ranges) throws TableNotFoundException,
       AccumuloException, AccumuloSecurityException {
-    
+
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 
     Instance instance = getInstance(job.getConfiguration());
     Connector conn = instance.getConnector(getUsername(job.getConfiguration()), getPassword(job.getConfiguration()));
     String tableId = Tables.getTableId(instance, tableName);
-    
+
     if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
       Tables.clearCache(instance);
       if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
@@ -1204,12 +1294,12 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
 
     for (Range range : ranges) {
       Text startRow;
-      
+
       if (range.getStartKey() != null)
         startRow = range.getStartKey().getRow();
       else
         startRow = new Text();
-      
+
       Range metadataRange = new Range(new KeyExtent(new Text(tableId), startRow, null).getMetadataEntry(), true, null, false);
       Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
       ColumnFQ.fetch(scanner, Constants.METADATA_PREV_ROW_COLUMN);
@@ -1217,9 +1307,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
       scanner.fetchColumnFamily(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY);
       scanner.setRange(metadataRange);
-      
+
       RowIterator rowIter = new RowIterator(scanner);
-      
+
       // TODO check that extents match prev extent
 
       KeyExtent lastExtent = null;
@@ -1229,15 +1319,15 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         String last = "";
         KeyExtent extent = null;
         String location = null;
-        
+
         while (row.hasNext()) {
           Entry<Key,Value> entry = row.next();
           Key key = entry.getKey();
-          
+
           if (key.getColumnFamily().equals(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY)) {
             last = entry.getValue().toString();
           }
-          
+
           if (key.getColumnFamily().equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY)
               || key.getColumnFamily().equals(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY)) {
             location = entry.getValue().toString();
@@ -1246,9 +1336,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           if (Constants.METADATA_PREV_ROW_COLUMN.hasColumns(key)) {
             extent = new KeyExtent(key.getRow(), entry.getValue());
           }
-          
+
         }
-        
+
         if (location != null)
           return null;
 
@@ -1265,24 +1355,24 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           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;
   }
 
@@ -1290,18 +1380,35 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * Read the metadata table to get tablets and match up ranges to them.
    */
   public List<InputSplit> getSplits(JobContext job) throws IOException {
-    log.setLevel(getLogLevel(job.getConfiguration()));
-    validateOptions(job.getConfiguration());
-    
-    String tableName = getTablename(job.getConfiguration());
-    boolean autoAdjust = getAutoAdjustRanges(job.getConfiguration());
-    List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(job.getConfiguration())) : getRanges(job.getConfiguration());
-    
+    Configuration conf = job.getConfiguration();
+
+    log.setLevel(getLogLevel(conf));
+    validateOptions(conf);
+
+    String tableName = getTablename(conf);
+    boolean autoAdjust = getAutoAdjustRanges(conf);
+    List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(conf)) : getRanges(conf);
+    boolean offline = isOfflineScan(conf);
+    boolean isolated = isIsolated(conf);
+    boolean localIterators = usesLocalIterators(conf);
+    boolean mockInstance = conf.getBoolean(MOCK, false);
+    int maxVersions = getMaxVersions(conf);
+    String rowRegex = conf.get(ROW_REGEX), colfamRegex = conf.get(COLUMN_FAMILY_REGEX), colqualRegex = conf.get(COLUMN_QUALIFIER_REGEX), valueRegex = conf
+        .get(VALUE_REGEX);
+    Set<Pair<Text,Text>> fetchedColumns = getFetchedColumns(conf);
+    Authorizations auths = getAuthorizations(conf);
+    byte[] password = getPassword(conf);
+    String username = getUsername(conf);
+    Instance instance = getInstance(conf);
+    List<AccumuloIterator> iterators = getIterators(conf);
+    List<AccumuloIteratorOption> options = getIteratorOptions(conf);
+    Level logLevel = getLogLevel(conf);
+
     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;
@@ -1314,7 +1421,6 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           binnedRanges = binOfflineTable(job, tableName, ranges);
         }
       } else {
-        Instance instance = getInstance(job.getConfiguration());
         String tableId = null;
         tl = getTabletLocator(job.getConfiguration());
         // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
@@ -1337,15 +1443,15 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     } catch (Exception e) {
       throw new IOException(e);
     }
-    
+
     ArrayList<InputSplit> splits = new ArrayList<InputSplit>(ranges.size());
     HashMap<Range,ArrayList<String>> splitsToAdd = null;
-    
+
     if (!autoAdjust)
       splitsToAdd = new HashMap<Range,ArrayList<String>>();
-    
+
     HashMap<String,String> hostNameCache = new HashMap<String,String>();
-    
+
     for (Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
       String ip = tserverBin.getKey().split(":", 2)[0];
       String location = hostNameCache.get(ip);
@@ -1354,14 +1460,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         location = inetAddress.getHostName();
         hostNameCache.put(ip, location);
       }
-      
+
       for (Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
         Range ke = extentRanges.getKey().toDataRange();
         for (Range r : extentRanges.getValue()) {
           if (autoAdjust) {
             // divide ranges into smaller ranges, based on the
             // tablets
-            splits.add(new RangeInputSplit(tableName, ke.clip(r), new String[] {location}));
+            splits.add(new RangeInputSplit(ke.clip(r), new String[] {location}));
           } else {
             // don't divide ranges
             ArrayList<String> locations = splitsToAdd.get(r);
@@ -1373,132 +1479,55 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         }
       }
     }
-    
+
     if (!autoAdjust)
       for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
-        splits.add(new RangeInputSplit(tableName, entry.getKey(), entry.getValue().toArray(new String[0])));
-    return splits;
-  }
-  
-  /**
-   * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
-   */
-  public static class RangeInputSplit extends InputSplit implements Writable {
-    private Range range;
-    private String[] locations;
-    
-    public RangeInputSplit() {
-      range = new Range();
-      locations = new String[0];
-    }
-    
-    public Range getRange() {
-      return range;
-    }
-    
-    private static byte[] extractBytes(ByteSequence seq, int numBytes) {
-      byte[] bytes = new byte[numBytes + 1];
-      bytes[0] = 0;
-      for (int i = 0; i < numBytes; i++) {
-        if (i >= seq.length())
-          bytes[i + 1] = 0;
-        else
-          bytes[i + 1] = seq.byteAt(i);
-      }
-      return bytes;
-    }
-    
-    public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
-      int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
-      BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
-      BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
-      BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
-      return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
-    }
-    
-    public float getProgress(Key currentKey) {
-      if (currentKey == null)
-        return 0f;
-      if (range.getStartKey() != null && range.getEndKey() != null) {
-        if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
-          // just look at the row progress
-          return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
-        } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
-          // just look at the column family progress
-          return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
-        } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL) != 0) {
-          // just look at the column qualifier progress
-          return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
-        }
-      }
-      // if we can't figure it out, then claim no progress
-      return 0f;
-    }
-    
-    RangeInputSplit(String table, Range range, String[] locations) {
-      this.range = range;
-      this.locations = locations;
-    }
-    
-    /**
-     * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
-     */
-    public long getLength() throws IOException {
-      Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
-      Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
-      int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
-      long diff = 0;
-      
-      byte[] start = startRow.getBytes();
-      byte[] stop = stopRow.getBytes();
-      for (int i = 0; i < maxCommon; ++i) {
-        diff |= 0xff & (start[i] ^ stop[i]);
-        diff <<= Byte.SIZE;
-      }
-      
-      if (startRow.getLength() != stopRow.getLength())
-        diff |= 0xff;
-      
-      return diff + 1;
-    }
-    
-    public String[] getLocations() throws IOException {
-      return locations;
-    }
-    
-    public void readFields(DataInput in) throws IOException {
-      range.readFields(in);
-      int numLocs = in.readInt();
-      locations = new String[numLocs];
-      for (int i = 0; i < numLocs; ++i)
-        locations[i] = in.readUTF();
-    }
-    
-    public void write(DataOutput out) throws IOException {
-      range.write(out);
-      out.writeInt(locations.length);
-      for (int i = 0; i < locations.length; ++i)
-        out.writeUTF(locations[i]);
+        splits.add(new RangeInputSplit(entry.getKey(), entry.getValue().toArray(new String[0])));
+
+    for (InputSplit inputSplit : splits) {
+      RangeInputSplit split = (RangeInputSplit) inputSplit;
+
+      split.setTable(tableName);
+      split.setOffline(offline);
+      split.setIsolatedScan(isolated);
+      split.setUsesLocalIterators(localIterators);
+      split.setMockInstance(mockInstance);
+      split.setMaxVersions(maxVersions);
+      split.setRowRegex(rowRegex);
+      split.setColfamRegex(colfamRegex);
+      split.setColqualRegex(colqualRegex);
+      split.setValueRegex(valueRegex);
+      split.setFetchedColumns(fetchedColumns);
+      split.setUsername(username);
+      split.setPassword(password);
+      split.setInstanceName(instance.getInstanceName());
+      split.setZooKeepers(instance.getZooKeepers());
+      split.setAuths(auths);
+      split.setIterators(iterators);
+      split.setOptions(options);
+      split.setLogLevel(logLevel);
     }
+
+    return splits;
   }
-  
+
   /**
    * The Class IteratorSetting. Encapsulates specifics for an Accumulo iterator's name & priority.
    */
   static class AccumuloIterator {
-    
+
     private static final String FIELD_SEP = ":";
-    
+
     private int priority;
     private String iteratorClass;
     private String iteratorName;
-    
+
     public AccumuloIterator(int priority, String iteratorClass, String iteratorName) {
       this.priority = priority;
       this.iteratorClass = iteratorClass;
       this.iteratorName = iteratorName;
     }
-    
+
     // Parses out a setting given an string supplied from an earlier toString() call
     public AccumuloIterator(String iteratorSetting) {
       // Parse the string to expand the iterator
@@ -1507,42 +1536,42 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       iteratorClass = tokenizer.nextToken();
       iteratorName = tokenizer.nextToken();
     }
-    
+
     public int getPriority() {
       return priority;
     }
-    
+
     public String getIteratorClass() {
       return iteratorClass;
     }
-    
+
     public String getIteratorName() {
       return iteratorName;
     }
-    
+
     @Override
     public String toString() {
       return new String(priority + FIELD_SEP + iteratorClass + FIELD_SEP + iteratorName);
     }
-    
+
   }
-  
+
   /**
    * The Class AccumuloIteratorOption. Encapsulates specifics for an Accumulo iterator's optional configuration details - associated via the iteratorName.
    */
   static class AccumuloIteratorOption {
     private static final String FIELD_SEP = ":";
-    
+
     private String iteratorName;
     private String key;
     private String value;
-    
+
     public AccumuloIteratorOption(String iteratorName, String key, String value) {
       this.iteratorName = iteratorName;
       this.key = key;
       this.value = value;
     }
-    
+
     // Parses out an option given a string supplied from an earlier toString() call
     public AccumuloIteratorOption(String iteratorOption) {
       StringTokenizer tokenizer = new StringTokenizer(iteratorOption, FIELD_SEP);
@@ -1554,19 +1583,19 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new RuntimeException(e);
       }
     }
-    
+
     public String getIteratorName() {
       return iteratorName;
     }
-    
+
     public String getKey() {
       return key;
     }
-    
+
     public String getValue() {
       return value;
     }
-    
+
     @Override
     public String toString() {
       try {
@@ -1575,7 +1604,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         throw new RuntimeException(e);
       }
     }
-    
+
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/45ae55fc/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
new file mode 100644
index 0000000..e372801
--- /dev/null
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -0,0 +1,493 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapreduce;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIterator;
+import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIteratorOption;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.PartialKey;
+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.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.Level;
+
+/**
+ * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
+ */
+public class RangeInputSplit extends InputSplit implements Writable {
+  private Range range;
+  private String[] locations;
+  private String table, instanceName, zooKeepers, username;
+  private String rowRegex, colfamRegex, colqualRegex, valueRegex;
+  private byte[] password;
+  private Boolean offline, mockInstance, isolatedScan, localIterators;
+  private Integer maxVersions;
+  private Authorizations auths;
+  private Set<Pair<Text,Text>> fetchedColumns;
+  private List<AccumuloIterator> iterators;
+  private List<AccumuloIteratorOption> options;
+  private Level level;
+
+  public RangeInputSplit() {
+    range = new Range();
+    locations = new String[0];
+  }
+
+  public RangeInputSplit(Range range, String[] locations) {
+    this.range = range;
+    this.locations = locations;
+  }
+
+  public Range getRange() {
+    return range;
+  }
+
+  private static byte[] extractBytes(ByteSequence seq, int numBytes) {
+    byte[] bytes = new byte[numBytes + 1];
+    bytes[0] = 0;
+    for (int i = 0; i < numBytes; i++) {
+      if (i >= seq.length())
+        bytes[i + 1] = 0;
+      else
+        bytes[i + 1] = seq.byteAt(i);
+    }
+    return bytes;
+  }
+
+  public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
+    int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
+    BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
+    BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
+    BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
+    return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
+  }
+
+  public float getProgress(Key currentKey) {
+    if (currentKey == null)
+      return 0f;
+    if (range.getStartKey() != null && range.getEndKey() != null) {
+      if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
+        // just look at the row progress
+        return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
+      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
+        // just look at the column family progress
+        return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
+      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL) != 0) {
+        // just look at the column qualifier progress
+        return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
+      }
+    }
+    // if we can't figure it out, then claim no progress
+    return 0f;
+  }
+
+  /**
+   * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
+   */
+  public long getLength() throws IOException {
+    Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
+    Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
+    int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
+    long diff = 0;
+
+    byte[] start = startRow.getBytes();
+    byte[] stop = stopRow.getBytes();
+    for (int i = 0; i < maxCommon; ++i) {
+      diff |= 0xff & (start[i] ^ stop[i]);
+      diff <<= Byte.SIZE;
+    }
+
+    if (startRow.getLength() != stopRow.getLength())
+      diff |= 0xff;
+
+    return diff + 1;
+  }
+
+  public String[] getLocations() throws IOException {
+    return locations;
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    range.readFields(in);
+    int numLocs = in.readInt();
+    locations = new String[numLocs];
+    for (int i = 0; i < numLocs; ++i)
+      locations[i] = in.readUTF();
+    
+    if (in.readBoolean()) {
+      isolatedScan = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      offline = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      localIterators = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      mockInstance = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      maxVersions = in.readInt();
+    }
+    
+    if (in.readBoolean()) {
+      rowRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      colfamRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      colqualRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      valueRegex = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      int numColumns = in.readInt();
+      String[] columns = new String[numColumns];
+      for (int i = 0; i < numColumns; i++) {
+        columns[i] = in.readUTF();
+      }
+      
+      fetchedColumns = InputFormatBase.deserializeFetchedColumns(columns);
+    }
+    
+    if (in.readBoolean()) {
+      auths = new Authorizations(StringUtils.split(in.readUTF()));
+    }
+    
+    if (in.readBoolean()) {
+      username = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      password = in.readUTF().getBytes();
+    }
+    
+    if (in.readBoolean()) {
+      instanceName = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      zooKeepers = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      level = Level.toLevel(in.readInt());
+    }
+  }
+
+  public void write(DataOutput out) throws IOException {
+    range.write(out);
+    out.writeInt(locations.length);
+    for (int i = 0; i < locations.length; ++i)
+      out.writeUTF(locations[i]);
+    
+    out.writeBoolean(null != isolatedScan);
+    if (null != isolatedScan) {
+      out.writeBoolean(isolatedScan);
+    }
+    
+    out.writeBoolean(null != offline);
+    if (null != offline) {
+      out.writeBoolean(offline);
+    }
+    
+    out.writeBoolean(null != localIterators);
+    if (null != localIterators) {
+      out.writeBoolean(localIterators);
+    }
+    
+    out.writeBoolean(null != mockInstance);
+    if (null != mockInstance) {
+      out.writeBoolean(mockInstance);
+    }
+    
+    out.writeBoolean(null != maxVersions);
+    if (null != maxVersions) {
+      out.writeInt(getMaxVersions());
+    }
+    
+    out.writeBoolean(null != rowRegex);
+    if (null != rowRegex) {
+      out.writeUTF(rowRegex);
+    }
+    
+    out.writeBoolean(null != colfamRegex);
+    if (null != colfamRegex) {
+      out.writeUTF(colfamRegex);
+    }
+    
+    out.writeBoolean(null != colqualRegex);
+    if (null != colqualRegex) {
+      out.writeUTF(colqualRegex);
+    }
+    
+    out.writeBoolean(null != valueRegex);
+    if (null != valueRegex) {
+      out.writeUTF(valueRegex);
+    }
+    
+    out.writeBoolean(null != fetchedColumns);
+    if (null != fetchedColumns) {
+      String[] cols = InputFormatBase.serializeColumns(fetchedColumns);
+      out.writeInt(cols.length);
+      for (String col : cols) {
+        out.writeUTF(col);
+      }
+    }
+    
+    out.writeBoolean(null != auths);
+    if (null != auths) {
+      out.writeUTF(auths.serialize());
+    }
+    
+    out.writeBoolean(null != username);
+    if (null != username) {
+      out.writeUTF(username);
+    }
+    
+    out.writeBoolean(null != password);
+    if (null != password) {
+      out.writeUTF(new String(password));
+    }
+    
+    out.writeBoolean(null != instanceName);
+    if (null != instanceName) {
+      out.writeUTF(instanceName);
+    }
+    
+    out.writeBoolean(null != zooKeepers);
+    if (null != zooKeepers) {
+      out.writeUTF(zooKeepers);
+    }
+    
+    out.writeBoolean(null != level);
+    if (null != level) {
+      out.writeInt(level.toInt());
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder(128);
+    sb.append("Range: ").append(range);
+    sb.append(" Locations: ").append(locations);
+    sb.append(" Table: ").append(table);
+    // TODO finish building of string
+    return sb.toString();
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+  
+  public Instance getInstance() {
+    if (null == instanceName) {
+      return null;
+    }
+    
+    if (isMockInstance()) {  
+      return new MockInstance(getInstanceName());
+    }
+    
+    if (null == zooKeepers) {
+      return null;
+    }
+    
+    return new ZooKeeperInstance(getInstanceName(), getZooKeepers());
+  }
+
+  public String getInstanceName() {
+    return instanceName;
+  }
+
+  public void setInstanceName(String instanceName) {
+    this.instanceName = instanceName;
+  }
+
+  public String getZooKeepers() {
+    return zooKeepers;
+  }
+
+  public void setZooKeepers(String zooKeepers) {
+    this.zooKeepers = zooKeepers;
+  }
+
+  public String getUsername() {
+    return username;
+  }
+
+  public void setUsername(String username) {
+    this.username = username;
+  }
+
+  public byte[] getPassword() {
+    return password;
+  }
+
+  public void setPassword(byte[] password) {
+    this.password = password;
+  }
+
+  public Boolean isOffline() {
+    return offline;
+  }
+
+  public void setOffline(Boolean offline) {
+    this.offline = offline;
+  }
+
+  public void setLocations(String[] locations) {
+    this.locations = locations;
+  }
+
+  public String getRowRegex() {
+    return rowRegex;
+  }
+
+  public void setRowRegex(String rowRegex) {
+    this.rowRegex = rowRegex;
+  }
+
+  public String getColfamRegex() {
+    return colfamRegex;
+  }
+
+  public void setColfamRegex(String colfamRegex) {
+    this.colfamRegex = colfamRegex;
+  }
+
+  public String getColqualRegex() {
+    return colqualRegex;
+  }
+
+  public void setColqualRegex(String colqualRegex) {
+    this.colqualRegex = colqualRegex;
+  }
+
+  public String getValueRegex() {
+    return valueRegex;
+  }
+
+  public void setValueRegex(String valueRegex) {
+    this.valueRegex = valueRegex;
+  }
+
+  public Boolean isMockInstance() {
+    return mockInstance;
+  }
+
+  public void setMockInstance(Boolean mockInstance) {
+    this.mockInstance = mockInstance;
+  }
+
+  public Boolean isIsolatedScan() {
+    return isolatedScan;
+  }
+
+  public void setIsolatedScan(Boolean isolatedScan) {
+    this.isolatedScan = isolatedScan;
+  }
+
+  public Integer getMaxVersions() {
+    return maxVersions;
+  }
+
+  public void setMaxVersions(Integer maxVersions) {
+    this.maxVersions = maxVersions;
+  }
+
+  public Authorizations getAuths() {
+    return auths;
+  }
+
+  public void setAuths(Authorizations auths) {
+    this.auths = auths;
+  }
+
+  public void setRange(Range range) {
+    this.range = range;
+  }
+
+  public Boolean usesLocalIterators() {
+    return localIterators;
+  }
+
+  public void setUsesLocalIterators(Boolean localIterators) {
+    this.localIterators = localIterators;
+  }
+
+  public Set<Pair<Text,Text>> getFetchedColumns() {
+    return fetchedColumns;
+  }
+
+  public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
+    this.fetchedColumns = fetchedColumns;
+  }
+
+  public List<AccumuloIterator> getIterators() {
+    return iterators;
+  }
+
+  public void setIterators(List<AccumuloIterator> iterators) {
+    this.iterators = iterators;
+  }
+
+  public List<AccumuloIteratorOption> getOptions() {
+    return options;
+  }
+
+  public void setOptions(List<AccumuloIteratorOption> options) {
+    this.options = options;
+  }
+  
+  public Level getLogLevel() {
+    return level;
+  }
+  
+  public void setLogLevel(Level level) {
+    this.level = level;
+  }
+}


[07/19] git commit: ACCUMULO-1854 Remove unnecessary arguments from methods

Posted by el...@apache.org.
ACCUMULO-1854 Remove unnecessary arguments from methods


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/3beb9f71
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/3beb9f71
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/3beb9f71

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 3beb9f710f526672fc67850b84c4999d968dc925
Parents: 45ae55f
Author: Josh Elser <el...@apache.org>
Authored: Thu Nov 21 20:18:16 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Nov 21 20:18:16 2013 -0500

----------------------------------------------------------------------
 .../core/client/mapreduce/InputFormatBase.java  | 34 ++++++--------------
 1 file changed, 10 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/3beb9f71/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index 40e09a1..bd90b8c 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -1057,25 +1057,18 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       }
     }
 
-    // Apply the configured iterators from the job to the scanner
-    /**
-     * @deprecated Use {@link #setupIterators(Configuration,Scanner)} instead
-     */
-    protected void setupIterators(TaskAttemptContext attempt, Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
-        throws AccumuloException {
-      setupIterators(attempt.getConfiguration(), scanner, iterators, options);
-    }
-
     /**
      * Apply the configured iterators from the configuration to the scanner.
      * 
-     * @param conf
-     *          the Hadoop configuration object
      * @param scanner
      *          the scanner to configure
+     * @param iterators
+     *          the iterators to configure on the scanner
+     * @param options
+     *          options for each configured iterator
      * @throws AccumuloException
      */
-    protected void setupIterators(Configuration conf, Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
+    protected void setupIterators(Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
         throws AccumuloException {
 
       Map<String,IteratorSetting> scanIterators = new HashMap<String,IteratorSetting>();
@@ -1091,21 +1084,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
 
     /**
-     * @deprecated Use {@link #setupMaxVersions(Configuration,Scanner)} instead
-     */
-    protected void setupMaxVersions(TaskAttemptContext attempt, Scanner scanner, int maxVersions) {
-      setupMaxVersions(attempt.getConfiguration(), scanner, maxVersions);
-    }
-
-    /**
      * If maxVersions has been set, configure a {@link VersioningIterator} at priority 0 for this scanner.
      * 
-     * @param conf
-     *          the Hadoop configuration object
      * @param scanner
      *          the scanner to configure
+     * @param maxVersions
+     *          the number of versions to return
      */
-    protected void setupMaxVersions(Configuration conf, Scanner scanner, int maxVersions) {
+    protected void setupMaxVersions(Scanner scanner, int maxVersions) {
       // Check to make sure its a legit value
       if (maxVersions >= 1) {
         IteratorSetting vers = new IteratorSetting(0, "vers", VersioningIterator.class);
@@ -1222,13 +1208,13 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           log.info("Using local iterators");
           scanner = new ClientSideIteratorScanner(scanner);
         }
-        setupMaxVersions(conf, scanner, maxVersions);
+        setupMaxVersions(scanner, maxVersions);
         if (rowRegex != null || colfRegex != null || colqRegex != null || valueRegex != null) {
           IteratorSetting is = new IteratorSetting(50, RegExFilter.class);
           RegExFilter.setRegexs(is, rowRegex, colfRegex, colqRegex, valueRegex, false);
           scanner.addScanIterator(is);
         }
-        setupIterators(conf, scanner, iterators, options);
+        setupIterators(scanner, iterators, options);
       } catch (Exception e) {
         throw new IOException(e);
       }


[15/19] Squashed commit of the following:

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/73114819/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
index ba647e9..7239b01 100644
--- a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
@@ -20,15 +20,17 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIterator;
 import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIteratorOption;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit;
 import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RegexType;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.data.Key;
@@ -36,6 +38,7 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -46,15 +49,16 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
 
 public class AccumuloInputFormatTest {
-  
+
   @After
   public void tearDown() throws Exception {}
-  
+
   /**
    * Test basic setting & getting of max versions.
    * 
@@ -68,7 +72,7 @@ public class AccumuloInputFormatTest {
     int version = AccumuloInputFormat.getMaxVersions(job.getConfiguration());
     assertEquals(1, version);
   }
-  
+
   /**
    * Test max versions with an invalid value.
    * 
@@ -80,7 +84,7 @@ public class AccumuloInputFormatTest {
     JobContext job = new JobContext(new Configuration(), new JobID());
     AccumuloInputFormat.setMaxVersions(job.getConfiguration(), 0);
   }
-  
+
   /**
    * Test no max version configured.
    */
@@ -89,7 +93,7 @@ public class AccumuloInputFormatTest {
     JobContext job = new JobContext(new Configuration(), new JobID());
     assertEquals(-1, AccumuloInputFormat.getMaxVersions(job.getConfiguration()));
   }
-  
+
   /**
    * Check that the iterator configuration is getting stored in the Job conf correctly.
    */
@@ -97,45 +101,45 @@ public class AccumuloInputFormatTest {
   @Test
   public void testSetIterator() {
     JobContext job = new JobContext(new Configuration(), new JobID());
-    
+
     AccumuloInputFormat.setIterator(job, 1, "org.apache.accumulo.core.iterators.WholeRowIterator", "WholeRow");
     Configuration conf = job.getConfiguration();
     String iterators = conf.get("AccumuloInputFormat.iterators");
     assertEquals("1:org.apache.accumulo.core.iterators.WholeRowIterator:WholeRow", iterators);
   }
-  
+
   @Test
   public void testAddIterator() {
     JobContext job = new JobContext(new Configuration(), new JobID());
-    
+
     AccumuloInputFormat.addIterator(job.getConfiguration(), new IteratorSetting(1, "WholeRow", WholeRowIterator.class));
     AccumuloInputFormat.addIterator(job.getConfiguration(), 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.getConfiguration(), iter);
-    
+
     List<AccumuloIterator> list = AccumuloInputFormat.getIterators(job.getConfiguration());
-    
+
     // Check the list size
     assertTrue(list.size() == 3);
-    
+
     // Walk the list and make sure our settings are correct
     AccumuloIterator setting = list.get(0);
     assertEquals(1, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.user.WholeRowIterator", setting.getIteratorClass());
     assertEquals("WholeRow", setting.getIteratorName());
-    
+
     setting = list.get(1);
     assertEquals(2, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
     assertEquals("Versions", setting.getIteratorName());
-    
+
     setting = list.get(2);
     assertEquals(3, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
     assertEquals("Count", setting.getIteratorName());
-    
+
     List<AccumuloIteratorOption> iteratorOptions = AccumuloInputFormat.getIteratorOptions(job.getConfiguration());
     assertEquals(2, iteratorOptions.size());
     assertEquals("Count", iteratorOptions.get(0).getIteratorName());
@@ -145,7 +149,7 @@ public class AccumuloInputFormatTest {
     assertEquals("junk", iteratorOptions.get(1).getKey());
     assertEquals("\0omg:!\\xyzzy", iteratorOptions.get(1).getValue());
   }
-  
+
   /**
    * 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.
@@ -160,16 +164,16 @@ public class AccumuloInputFormatTest {
     someSetting.addOption(key, value);
     Job job = new Job();
     AccumuloInputFormat.addIterator(job.getConfiguration(), someSetting);
-    
+
     final String rawConfigOpt = new AccumuloIteratorOption("iterator", key, value).toString();
-    
+
     assertEquals(rawConfigOpt, job.getConfiguration().get("AccumuloInputFormat.iterators.options"));
-    
+
     List<AccumuloIteratorOption> opts = AccumuloInputFormat.getIteratorOptions(job.getConfiguration());
     assertEquals(1, opts.size());
     assertEquals(opts.get(0).getKey(), key);
     assertEquals(opts.get(0).getValue(), value);
-    
+
     someSetting.addOption(key + "2", value);
     someSetting.setPriority(2);
     someSetting.setName("it2");
@@ -181,7 +185,7 @@ public class AccumuloInputFormatTest {
       assertEquals(opt.getValue(), value);
     }
   }
-  
+
   /**
    * Test getting iterator settings for multiple iterators set
    */
@@ -189,34 +193,34 @@ public class AccumuloInputFormatTest {
   @Test
   public void testGetIteratorSettings() {
     JobContext job = new JobContext(new Configuration(), new JobID());
-    
+
     AccumuloInputFormat.setIterator(job, 1, "org.apache.accumulo.core.iterators.WholeRowIterator", "WholeRow");
     AccumuloInputFormat.setIterator(job, 2, "org.apache.accumulo.core.iterators.VersioningIterator", "Versions");
     AccumuloInputFormat.setIterator(job, 3, "org.apache.accumulo.core.iterators.CountingIterator", "Count");
-    
+
     List<AccumuloIterator> list = AccumuloInputFormat.getIterators(job);
-    
+
     // Check the list size
     assertTrue(list.size() == 3);
-    
+
     // Walk the list and make sure our settings are correct
     AccumuloIterator setting = list.get(0);
     assertEquals(1, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.WholeRowIterator", setting.getIteratorClass());
     assertEquals("WholeRow", setting.getIteratorName());
-    
+
     setting = list.get(1);
     assertEquals(2, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
     assertEquals("Versions", setting.getIteratorName());
-    
+
     setting = list.get(2);
     assertEquals(3, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
     assertEquals("Count", setting.getIteratorName());
-    
+
   }
-  
+
   /**
    * Check that the iterator options are getting stored in the Job conf correctly.
    */
@@ -225,12 +229,12 @@ public class AccumuloInputFormatTest {
   public void testSetIteratorOption() {
     JobContext job = new JobContext(new Configuration(), new JobID());
     AccumuloInputFormat.setIteratorOption(job, "someIterator", "aKey", "aValue");
-    
+
     Configuration conf = job.getConfiguration();
     String options = conf.get("AccumuloInputFormat.iterators.options");
     assertEquals(new String("someIterator:aKey:aValue"), options);
   }
-  
+
   /**
    * Test getting iterator options for multiple options set
    */
@@ -238,49 +242,49 @@ public class AccumuloInputFormatTest {
   @Test
   public void testGetIteratorOption() {
     JobContext job = new JobContext(new Configuration(), new JobID());
-    
+
     AccumuloInputFormat.setIteratorOption(job, "iterator1", "key1", "value1");
     AccumuloInputFormat.setIteratorOption(job, "iterator2", "key2", "value2");
     AccumuloInputFormat.setIteratorOption(job, "iterator3", "key3", "value3");
-    
+
     List<AccumuloIteratorOption> list = AccumuloInputFormat.getIteratorOptions(job);
-    
+
     // Check the list size
     assertEquals(3, list.size());
-    
+
     // Walk the list and make sure all the options are correct
     AccumuloIteratorOption option = list.get(0);
     assertEquals("iterator1", option.getIteratorName());
     assertEquals("key1", option.getKey());
     assertEquals("value1", option.getValue());
-    
+
     option = list.get(1);
     assertEquals("iterator2", option.getIteratorName());
     assertEquals("key2", option.getKey());
     assertEquals("value2", option.getValue());
-    
+
     option = list.get(2);
     assertEquals("iterator3", option.getIteratorName());
     assertEquals("key3", option.getKey());
     assertEquals("value3", option.getValue());
   }
-  
+
   @SuppressWarnings("deprecation")
   @Test
   public void testSetRegex() {
     JobContext job = new JobContext(new Configuration(), new JobID());
-    
+
     String regex = ">\"*%<>\'\\";
-    
+
     AccumuloInputFormat.setRegex(job, RegexType.ROW, regex);
-    
+
     assertTrue(regex.equals(AccumuloInputFormat.getRegex(job, RegexType.ROW)));
   }
-  
+
   static class TestMapper extends Mapper<Key,Value,Key,Value> {
     Key key = null;
     int count = 0;
-    
+
     @Override
     protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
       if (key != null)
@@ -291,7 +295,7 @@ public class AccumuloInputFormatTest {
       count++;
     }
   }
-  
+
   @Test
   public void testMap() throws Exception {
     MockInstance mockInstance = new MockInstance("testmapinstance");
@@ -304,20 +308,27 @@ public class AccumuloInputFormatTest {
       bw.addMutation(m);
     }
     bw.close();
-    
+
     Job job = new Job(new Configuration());
     job.setInputFormatClass(AccumuloInputFormat.class);
     job.setMapperClass(TestMapper.class);
     job.setNumReduceTasks(0);
     AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
     AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
-    
+
     AccumuloInputFormat input = new AccumuloInputFormat();
     List<InputSplit> splits = input.getSplits(job);
     assertEquals(splits.size(), 1);
-    
+
     TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
     for (InputSplit split : splits) {
+      RangeInputSplit risplit = (RangeInputSplit) split;
+      Assert.assertEquals("root", risplit.getUsername());
+      Assert.assertArrayEquals(new byte[0], risplit.getPassword());
+      Assert.assertEquals("testtable", risplit.getTable());
+      Assert.assertEquals(new Authorizations(), risplit.getAuths());
+      Assert.assertEquals("testmapinstance", risplit.getInstanceName());
+      
       TaskAttemptID id = new TaskAttemptID();
       TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
       RecordReader<Key,Value> reader = input.createRecordReader(split, attempt);
@@ -326,7 +337,7 @@ public class AccumuloInputFormatTest {
       mapper.run(context);
     }
   }
-  
+
   @Test
   public void testSimple() throws Exception {
     MockInstance mockInstance = new MockInstance("testmapinstance");
@@ -339,7 +350,7 @@ public class AccumuloInputFormatTest {
       bw.addMutation(m);
     }
     bw.close();
-    
+
     JobContext job = new JobContext(new Configuration(), new JobID());
     AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable2", new Authorizations());
     AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
@@ -348,14 +359,14 @@ public class AccumuloInputFormatTest {
     TaskAttemptContext tac = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID());
     RecordReader<Key,Value> rr = input.createRecordReader(ris, tac);
     rr.initialize(ris, tac);
-    
+
     TestMapper mapper = new TestMapper();
     Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), tac.getTaskAttemptID(), rr, null, null, null, ris);
     while (rr.nextKeyValue()) {
       mapper.map(rr.getCurrentKey(), rr.getCurrentValue(), context);
     }
   }
-  
+
   @SuppressWarnings("deprecation")
   @Test
   public void testRegex() throws Exception {
@@ -369,7 +380,7 @@ public class AccumuloInputFormatTest {
       bw.addMutation(m);
     }
     bw.close();
-    
+
     JobContext job = new JobContext(new Configuration(), new JobID());
     AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable3", new Authorizations());
     AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
@@ -380,10 +391,144 @@ public class AccumuloInputFormatTest {
     TaskAttemptContext tac = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID());
     RecordReader<Key,Value> rr = input.createRecordReader(ris, tac);
     rr.initialize(ris, tac);
-    
+
     Pattern p = Pattern.compile(regex);
     while (rr.nextKeyValue()) {
-      Assert.assertTrue( p.matcher( rr.getCurrentKey().getRow().toString()).matches());
+      Assert.assertTrue(p.matcher(rr.getCurrentKey().getRow().toString()).matches());
+    }
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testCorrectRangeInputSplits() throws Exception {
+    JobContext job = new JobContext(new Configuration(), new JobID());
+
+    String username = "user", table = "table", rowRegex = "row.*", colfRegex = "colf.*", colqRegex = "colq.*";
+    String valRegex = "val.*", instance = "instance";
+    byte[] password = "password".getBytes();
+    Authorizations auths = new Authorizations("foo");
+    Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
+    boolean isolated = true, localIters = true;
+    int maxVersions = 5;
+    Level level = Level.WARN;
+
+    Instance inst = new MockInstance(instance);
+    Connector connector = inst.getConnector(username, password);
+    connector.tableOperations().create(table);
+
+    AccumuloInputFormat.setInputInfo(job, username, password, table, auths);
+    AccumuloInputFormat.setMockInstance(job, instance);
+    AccumuloInputFormat.setRegex(job, RegexType.ROW, rowRegex);
+    AccumuloInputFormat.setRegex(job, RegexType.COLUMN_FAMILY, colfRegex);
+    AccumuloInputFormat.setRegex(job, RegexType.COLUMN_QUALIFIER, colqRegex);
+    AccumuloInputFormat.setRegex(job, RegexType.VALUE, valRegex);
+    AccumuloInputFormat.setIsolated(job, isolated);
+    AccumuloInputFormat.setLocalIterators(job, localIters);
+    AccumuloInputFormat.setMaxVersions(job, maxVersions);
+    AccumuloInputFormat.fetchColumns(job, fetchColumns);
+    AccumuloInputFormat.setLogLevel(job, level);
+    
+    AccumuloInputFormat aif = new AccumuloInputFormat();
+    
+    List<InputSplit> splits = aif.getSplits(job);
+    
+    Assert.assertEquals(1, splits.size());
+    
+    InputSplit split = splits.get(0);
+    
+    Assert.assertEquals(RangeInputSplit.class, split.getClass());
+    
+    RangeInputSplit risplit = (RangeInputSplit) split;
+    
+    Assert.assertEquals(username, risplit.getUsername());
+    Assert.assertEquals(table, risplit.getTable());
+    Assert.assertArrayEquals(password, risplit.getPassword());
+    Assert.assertEquals(auths, risplit.getAuths());
+    Assert.assertEquals(instance, risplit.getInstanceName());
+    Assert.assertEquals(rowRegex, risplit.getRowRegex());
+    Assert.assertEquals(colfRegex, risplit.getColfamRegex());
+    Assert.assertEquals(colqRegex, risplit.getColqualRegex());
+    Assert.assertEquals(valRegex, risplit.getValueRegex());
+    Assert.assertEquals(isolated, risplit.isIsolatedScan());
+    Assert.assertEquals(localIters, risplit.usesLocalIterators());
+    Assert.assertEquals(maxVersions, risplit.getMaxVersions().intValue());
+    Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());
+    Assert.assertEquals(level, risplit.getLogLevel());
+  }
+
+  @Test
+  public void testPartialInputSplitDelegationToConfiguration() throws Exception {
+    MockInstance mockInstance = new MockInstance("testPartialInputSplitDelegationToConfiguration");
+    Connector c = mockInstance.getConnector("root", new byte[] {});
+    c.tableOperations().create("testtable");
+    BatchWriter bw = c.createBatchWriter("testtable", 10000L, 1000L, 4);
+    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();
+
+    Job job = new Job(new Configuration());
+    job.setInputFormatClass(AccumuloInputFormat.class);
+    job.setMapperClass(TestMapper.class);
+    job.setNumReduceTasks(0);
+    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
+    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testPartialInputSplitDelegationToConfiguration");
+
+    AccumuloInputFormat input = new AccumuloInputFormat();
+    List<InputSplit> splits = input.getSplits(job);
+    assertEquals(splits.size(), 1);
+
+    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
+    
+    RangeInputSplit emptySplit = new RangeInputSplit();
+    
+    // Using an empty split should fall back to the information in the Job's Configuration
+    TaskAttemptID id = new TaskAttemptID();
+    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
+    RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
+    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
+    reader.initialize(emptySplit, context);
+    mapper.run(context);
+  }
+
+  @Test(expected = IOException.class)
+  public void testPartialFailedInputSplitDelegationToConfiguration() throws Exception {
+    MockInstance mockInstance = new MockInstance("testPartialFailedInputSplitDelegationToConfiguration");
+    Connector c = mockInstance.getConnector("root", new byte[] {});
+    c.tableOperations().create("testtable");
+    BatchWriter bw = c.createBatchWriter("testtable", 10000L, 1000L, 4);
+    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();
+
+    Job job = new Job(new Configuration());
+    job.setInputFormatClass(AccumuloInputFormat.class);
+    job.setMapperClass(TestMapper.class);
+    job.setNumReduceTasks(0);
+    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
+    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testPartialFailedInputSplitDelegationToConfiguration");
+
+    AccumuloInputFormat input = new AccumuloInputFormat();
+    List<InputSplit> splits = input.getSplits(job);
+    assertEquals(splits.size(), 1);
+
+    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
+    
+    RangeInputSplit emptySplit = new RangeInputSplit();
+    emptySplit.setUsername("root");
+    emptySplit.setPassword("anythingelse".getBytes());
+    
+    // Using an empty split should fall back to the information in the Job's Configuration
+    TaskAttemptID id = new TaskAttemptID();
+    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
+    RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
+    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
+    reader.initialize(emptySplit, context);
+    mapper.run(context);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/73114819/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
index 0673f1b..d9f9da0 100644
--- a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
@@ -27,7 +27,6 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/73114819/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
----------------------------------------------------------------------
diff --git a/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
new file mode 100644
index 0000000..22fb6e1
--- /dev/null
+++ b/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
@@ -0,0 +1,100 @@
+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.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(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.getRange(), newSplit.getRange());
+    Assert.assertTrue(Arrays.equals(split.getLocations(), newSplit.getLocations()));
+  }
+
+
+
+  @Test
+  public void testAllFieldsWritable() throws IOException {
+    RangeInputSplit split = new RangeInputSplit(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.setMaxVersions(5);
+    split.setRowRegex("row");
+    split.setColfamRegex("colf");
+    split.setColqualRegex("colq");
+    split.setValueRegex("value");
+    split.setFetchedColumns(fetchedColumns);
+    split.setPassword("password".getBytes());
+    split.setUsername("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.getMaxVersions(), newSplit.getMaxVersions());
+    Assert.assertEquals(split.getRowRegex(), newSplit.getRowRegex());
+    Assert.assertEquals(split.getColfamRegex(), newSplit.getColfamRegex());
+    Assert.assertEquals(split.getColqualRegex(), newSplit.getColqualRegex());
+    Assert.assertEquals(split.getValueRegex(), newSplit.getValueRegex());
+    Assert.assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
+    Assert.assertEquals(new String(split.getPassword()), new String(newSplit.getPassword()));
+    Assert.assertEquals(split.getUsername(), newSplit.getUsername());
+    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/73114819/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
----------------------------------------------------------------------
diff --git a/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java b/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
index c31c738..af12302 100644
--- a/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
+++ b/src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
@@ -30,15 +30,13 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit;
+import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
 import org.apache.accumulo.core.client.mock.MockInstance;
 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.ColumnVisibility;
-import org.apache.accumulo.examples.simple.filedata.ChunkInputFormat;
-import org.apache.accumulo.examples.simple.filedata.ChunkInputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobID;


[03/19] git commit: ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation

Posted by el...@apache.org.
ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/2640ea9d
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/2640ea9d
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/2640ea9d

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 2640ea9d926f771274ed10efd23e7400a483f6c0
Parents: 3f633bf
Author: Eric Newton <er...@gmail.com>
Authored: Thu Nov 21 12:19:01 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Nov 21 12:55:51 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/server/util/MetadataTable.java  | 12 +++++++-----
 1 file changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2640ea9d/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
----------------------------------------------------------------------
diff --git a/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java b/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
index 9c4eebf..a9b72e0 100644
--- a/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
+++ b/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
@@ -991,8 +991,8 @@ public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
   }
   
   public static void removeUnusedWALEntries(KeyExtent extent, List<LogEntry> logEntries, ZooLock zooLock) {
-    for (LogEntry entry : logEntries) {
-      if (entry.extent.equals(Constants.ROOT_TABLET_EXTENT)) {
+      if (extent.equals(Constants.ROOT_TABLET_EXTENT)) {
+        for (LogEntry entry : logEntries) {
         String root = getZookeeperLogLocation();
         while (true) {
           try {
@@ -1005,11 +1005,13 @@ public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
           }
           UtilWaitThread.sleep(1000);
         }
+        }
       } else {
-        Mutation m = new Mutation(entry.extent.getMetadataEntry());
-        m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename));
+        Mutation m = new Mutation(extent.getMetadataEntry());
+        for (LogEntry entry : logEntries) {
+          m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename));
+        }
         update(SecurityConstants.getSystemCredentials(), zooLock, m);
-      }
     }
   }
   


[17/19] git commit: ACCUMULO-1854 Remove unnecessary arguments from methods

Posted by el...@apache.org.
ACCUMULO-1854 Remove unnecessary arguments from methods


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/8dd3ae4b
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/8dd3ae4b
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/8dd3ae4b

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 8dd3ae4b1d614182f26d382d3d16956726d26702
Parents: 7311481
Author: Josh Elser <el...@apache.org>
Authored: Thu Nov 21 20:18:16 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Nov 22 15:17:58 2013 -0500

----------------------------------------------------------------------
 .../core/client/mapreduce/InputFormatBase.java  | 34 ++++++--------------
 1 file changed, 10 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/8dd3ae4b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index 40e09a1..bd90b8c 100644
--- a/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -1057,25 +1057,18 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       }
     }
 
-    // Apply the configured iterators from the job to the scanner
-    /**
-     * @deprecated Use {@link #setupIterators(Configuration,Scanner)} instead
-     */
-    protected void setupIterators(TaskAttemptContext attempt, Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
-        throws AccumuloException {
-      setupIterators(attempt.getConfiguration(), scanner, iterators, options);
-    }
-
     /**
      * Apply the configured iterators from the configuration to the scanner.
      * 
-     * @param conf
-     *          the Hadoop configuration object
      * @param scanner
      *          the scanner to configure
+     * @param iterators
+     *          the iterators to configure on the scanner
+     * @param options
+     *          options for each configured iterator
      * @throws AccumuloException
      */
-    protected void setupIterators(Configuration conf, Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
+    protected void setupIterators(Scanner scanner, List<AccumuloIterator> iterators, List<AccumuloIteratorOption> options)
         throws AccumuloException {
 
       Map<String,IteratorSetting> scanIterators = new HashMap<String,IteratorSetting>();
@@ -1091,21 +1084,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
 
     /**
-     * @deprecated Use {@link #setupMaxVersions(Configuration,Scanner)} instead
-     */
-    protected void setupMaxVersions(TaskAttemptContext attempt, Scanner scanner, int maxVersions) {
-      setupMaxVersions(attempt.getConfiguration(), scanner, maxVersions);
-    }
-
-    /**
      * If maxVersions has been set, configure a {@link VersioningIterator} at priority 0 for this scanner.
      * 
-     * @param conf
-     *          the Hadoop configuration object
      * @param scanner
      *          the scanner to configure
+     * @param maxVersions
+     *          the number of versions to return
      */
-    protected void setupMaxVersions(Configuration conf, Scanner scanner, int maxVersions) {
+    protected void setupMaxVersions(Scanner scanner, int maxVersions) {
       // Check to make sure its a legit value
       if (maxVersions >= 1) {
         IteratorSetting vers = new IteratorSetting(0, "vers", VersioningIterator.class);
@@ -1222,13 +1208,13 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           log.info("Using local iterators");
           scanner = new ClientSideIteratorScanner(scanner);
         }
-        setupMaxVersions(conf, scanner, maxVersions);
+        setupMaxVersions(scanner, maxVersions);
         if (rowRegex != null || colfRegex != null || colqRegex != null || valueRegex != null) {
           IteratorSetting is = new IteratorSetting(50, RegExFilter.class);
           RegExFilter.setRegexs(is, rowRegex, colfRegex, colqRegex, valueRegex, false);
           scanner.addScanIterator(is);
         }
-        setupIterators(conf, scanner, iterators, options);
+        setupIterators(scanner, iterators, options);
       } catch (Exception e) {
         throw new IOException(e);
       }


[19/19] git commit: Merge branch '1.4.5-SNAPSHOT' into ACCUMULO-1854-1.5-merge

Posted by el...@apache.org.
Merge branch '1.4.5-SNAPSHOT' into ACCUMULO-1854-1.5-merge

Conflicts:
	core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
	server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
	src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
	src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
	src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
	src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/3a703d75
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/3a703d75
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/3a703d75

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 3a703d75d0bc37ad7aceb7357f1a69b67b7216d4
Parents: 6718920 e1dd6f9
Author: Josh Elser <el...@apache.org>
Authored: Fri Nov 22 20:29:48 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Nov 22 20:29:48 2013 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------



[05/19] WIP Merge branch 'ACCUMULO-1854-merge' into ACCUMULO-1854-1.5-merge

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/16a2e0f0/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index 86ef711,0000000..e17b46d
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@@ -1,1359 -1,0 +1,1293 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.accumulo.core.client.mapreduce;
 +
 +import java.io.DataInput;
 +import java.io.DataOutput;
 +import java.io.IOException;
 +import java.io.UnsupportedEncodingException;
 +import java.lang.reflect.Method;
 +import java.math.BigInteger;
 +import java.net.InetAddress;
 +import java.net.URLDecoder;
 +import java.net.URLEncoder;
 +import java.nio.ByteBuffer;
++import java.nio.charset.Charset;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.HashMap;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.StringTokenizer;
 +
 +import org.apache.accumulo.core.Constants;
 +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.TableDeletedException;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.TableOfflineException;
 +import org.apache.accumulo.core.client.ZooKeeperInstance;
 +import org.apache.accumulo.core.client.impl.OfflineScanner;
 +import org.apache.accumulo.core.client.impl.Tables;
 +import org.apache.accumulo.core.client.impl.TabletLocator;
 +import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 +import org.apache.accumulo.core.client.mock.MockInstance;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.data.ByteSequence;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.iterators.user.VersioningIterator;
 +import org.apache.accumulo.core.master.state.tables.TableState;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.io.Writable;
 +import org.apache.hadoop.mapreduce.InputFormat;
 +import org.apache.hadoop.mapreduce.InputSplit;
 +import org.apache.hadoop.mapreduce.Job;
 +import org.apache.hadoop.mapreduce.JobContext;
 +import org.apache.hadoop.mapreduce.RecordReader;
 +import org.apache.hadoop.mapreduce.TaskAttemptContext;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of K,V pairs.
 + * <p>
 + * Subclasses must implement a {@link #createRecordReader(InputSplit, TaskAttemptContext)} to provide a {@link RecordReader} for K,V.
 + * <p>
 + * A static base class, RecordReaderBase, is provided to retrieve Accumulo {@link Key}/{@link Value} pairs, but one must implement its
 + * {@link RecordReaderBase#nextKeyValue()} to transform them to the desired generic types K,V.
 + * <p>
 + * See {@link AccumuloInputFormat} for an example implementation.
 + */
 +public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
 +  
 +  private static final Class<?> CLASS = AccumuloInputFormat.class;
 +  protected static final Logger log = Logger.getLogger(CLASS);
 +  
 +  /**
 +   * Sets the connector information needed to communicate with Accumulo in this job.
 +   * 
 +   * <p>
 +   * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to provide a charset safe
 +   * conversion to a string, and is not intended to be secure.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param principal
 +   *          a valid Accumulo user name (user must have Table.CREATE permission)
 +   * @param token
 +   *          the user's password
 +   * @throws AccumuloSecurityException
 +   * @since 1.5.0
 +   */
 +  public static void setConnectorInfo(Job job, String principal, AuthenticationToken token) throws AccumuloSecurityException {
 +    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
 +  }
 +  
 +  /**
 +   * Determines if the connector has been configured.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return true if the connector has been configured, false otherwise
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(Job, String, AuthenticationToken)
 +   */
 +  protected static Boolean isConnectorInfoSet(JobContext context) {
 +    return InputConfigurator.isConnectorInfoSet(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Gets the user name from the configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return the user name
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(Job, String, AuthenticationToken)
 +   */
 +  protected static String getPrincipal(JobContext context) {
 +    return InputConfigurator.getPrincipal(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Gets the serialized token class from the configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return the user name
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(Job, String, AuthenticationToken)
 +   */
 +  protected static String getTokenClass(JobContext context) {
 +    return InputConfigurator.getTokenClass(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Gets the password from the configuration. WARNING: The password 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 context
 +   *          the Hadoop context for the configured job
 +   * @return the decoded user password
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(Job, String, AuthenticationToken)
 +   */
 +  protected static byte[] getToken(JobContext context) {
 +    return InputConfigurator.getToken(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Configures a {@link ZooKeeperInstance} for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param instanceName
 +   *          the Accumulo instance name
 +   * @param zooKeepers
 +   *          a comma-separated list of zookeeper servers
 +   * @since 1.5.0
 +   */
 +  public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
 +    InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), instanceName, zooKeepers);
 +  }
 +  
 +  /**
 +   * Configures a {@link MockInstance} for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param instanceName
 +   *          the Accumulo instance name
 +   * @since 1.5.0
 +   */
 +  public static void setMockInstance(Job job, String instanceName) {
 +    InputConfigurator.setMockInstance(CLASS, job.getConfiguration(), instanceName);
 +  }
 +  
 +  /**
 +   * Initializes an Accumulo {@link Instance} based on the configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return an Accumulo instance
 +   * @since 1.5.0
 +   * @see #setZooKeeperInstance(Job, String, String)
 +   * @see #setMockInstance(Job, String)
 +   */
 +  protected static Instance getInstance(JobContext context) {
 +    return InputConfigurator.getInstance(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Sets the log level for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param level
 +   *          the logging level
 +   * @since 1.5.0
 +   */
 +  public static void setLogLevel(Job job, Level level) {
 +    InputConfigurator.setLogLevel(CLASS, job.getConfiguration(), level);
 +  }
 +  
 +  /**
 +   * Gets the log level from this configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return the log level
 +   * @since 1.5.0
 +   * @see #setLogLevel(Job, Level)
 +   */
 +  protected static Level getLogLevel(JobContext context) {
 +    return InputConfigurator.getLogLevel(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Sets the name of the input table, over which this job will scan.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param tableName
 +   *          the table to use when the tablename is null in the write call
 +   * @since 1.5.0
 +   */
 +  public static void setInputTableName(Job job, String tableName) {
 +    InputConfigurator.setInputTableName(CLASS, job.getConfiguration(), tableName);
 +  }
 +  
 +  /**
 +   * Gets the table name from the configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return the table name
 +   * @since 1.5.0
 +   * @see #setInputTableName(Job, String)
 +   */
 +  protected static String getInputTableName(JobContext context) {
 +    return InputConfigurator.getInputTableName(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Sets the {@link 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(Job job, Authorizations auths) {
 +    InputConfigurator.setScanAuthorizations(CLASS, job.getConfiguration(), auths);
 +  }
 +  
 +  /**
 +   * Gets the authorizations to set for the scans from the configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return the Accumulo scan authorizations
 +   * @since 1.5.0
 +   * @see #setScanAuthorizations(Job, Authorizations)
 +   */
 +  protected static Authorizations getScanAuthorizations(JobContext context) {
 +    return InputConfigurator.getScanAuthorizations(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Sets the input ranges to scan for this job. If not set, the entire table will be scanned.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param ranges
 +   *          the ranges that will be mapped over
 +   * @since 1.5.0
 +   */
 +  public static void setRanges(Job job, Collection<Range> ranges) {
 +    InputConfigurator.setRanges(CLASS, job.getConfiguration(), ranges);
 +  }
 +  
 +  /**
 +   * Gets the ranges to scan over from a job.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return the ranges
 +   * @throws IOException
 +   *           if the ranges have been encoded improperly
 +   * @since 1.5.0
 +   * @see #setRanges(Job, Collection)
 +   */
 +  protected static List<Range> getRanges(JobContext context) throws IOException {
 +    return InputConfigurator.getRanges(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Restricts the columns that will be mapped over for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param columnFamilyColumnQualifierPairs
 +   *          a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
 +   *          selected. An empty set is the default and is equivalent to scanning the all columns.
 +   * @since 1.5.0
 +   */
 +  public static void fetchColumns(Job job, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 +    InputConfigurator.fetchColumns(CLASS, job.getConfiguration(), columnFamilyColumnQualifierPairs);
 +  }
 +  
 +  /**
 +   * Gets the columns to be mapped over from this job.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return a set of columns
 +   * @since 1.5.0
 +   * @see #fetchColumns(Job, Collection)
 +   */
 +  protected static Set<Pair<Text,Text>> getFetchedColumns(JobContext context) {
 +    return InputConfigurator.getFetchedColumns(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Encode an iterator on the input for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param cfg
 +   *          the configuration of the iterator
 +   * @since 1.5.0
 +   */
 +  public static void addIterator(Job job, IteratorSetting cfg) {
 +    InputConfigurator.addIterator(CLASS, job.getConfiguration(), cfg);
 +  }
 +  
 +  /**
 +   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return a list of iterators
 +   * @since 1.5.0
 +   * @see #addIterator(Job, IteratorSetting)
 +   */
 +  protected static List<IteratorSetting> getIterators(JobContext context) {
 +    return InputConfigurator.getIterators(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
 +   * Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
 +   * 
 +   * <p>
 +   * By default, this feature is <b>enabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @see #setRanges(Job, Collection)
 +   * @since 1.5.0
 +   */
 +  public static void setAutoAdjustRanges(Job job, boolean enableFeature) {
 +    InputConfigurator.setAutoAdjustRanges(CLASS, job.getConfiguration(), enableFeature);
 +  }
 +  
 +  /**
 +   * Determines whether a configuration has auto-adjust ranges enabled.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return false if the feature is disabled, true otherwise
 +   * @since 1.5.0
 +   * @see #setAutoAdjustRanges(Job, boolean)
 +   */
 +  protected static boolean getAutoAdjustRanges(JobContext context) {
 +    return InputConfigurator.getAutoAdjustRanges(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Controls the use of the {@link IsolatedScanner} in this job.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.5.0
 +   */
 +  public static void setScanIsolation(Job job, boolean enableFeature) {
 +    InputConfigurator.setScanIsolation(CLASS, job.getConfiguration(), enableFeature);
 +  }
 +  
 +  /**
 +   * Determines whether a configuration has isolation enabled.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.5.0
 +   * @see #setScanIsolation(Job, boolean)
 +   */
 +  protected static boolean isIsolated(JobContext context) {
 +    return InputConfigurator.isIsolated(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack to be constructed within the Map
 +   * task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be available on the classpath for the task.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.5.0
 +   */
 +  public static void setLocalIterators(Job job, boolean enableFeature) {
 +    InputConfigurator.setLocalIterators(CLASS, job.getConfiguration(), enableFeature);
 +  }
 +  
 +  /**
 +   * Determines whether a configuration uses local iterators.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.5.0
 +   * @see #setLocalIterators(Job, boolean)
 +   */
 +  protected static boolean usesLocalIterators(JobContext context) {
 +    return InputConfigurator.usesLocalIterators(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * <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. The accumulo-site.xml may need to be on the mapper's classpath if HDFS or the Accumulo directory in HDFS are non-standard.
 +   * 
 +   * <p>
 +   * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
 +   * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
 +   * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
 +   * 
 +   * <p>
 +   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
 +   * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.5.0
 +   */
 +  public static void setOfflineTableScan(Job job, boolean enableFeature) {
 +    InputConfigurator.setOfflineTableScan(CLASS, job.getConfiguration(), enableFeature);
 +  }
 +  
 +  /**
 +   * Determines whether a configuration has the offline table scan feature enabled.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.5.0
 +   * @see #setOfflineTableScan(Job, boolean)
 +   */
 +  protected static boolean isOfflineScan(JobContext context) {
 +    return InputConfigurator.isOfflineScan(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * Initializes an Accumulo {@link TabletLocator} based on the configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return an Accumulo tablet locator
 +   * @throws TableNotFoundException
 +   *           if the table name set on the configuration doesn't exist
 +   * @since 1.5.0
 +   */
 +  protected static TabletLocator getTabletLocator(JobContext context) throws TableNotFoundException {
 +    return InputConfigurator.getTabletLocator(CLASS, getConfiguration(context));
 +  }
 +  
 +  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
 +  /**
 +   * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @throws IOException
 +   *           if the context is improperly configured
 +   * @since 1.5.0
 +   */
 +  protected static void validateOptions(JobContext context) throws IOException {
 +    InputConfigurator.validateOptions(CLASS, getConfiguration(context));
 +  }
 +  
 +  /**
 +   * An abstract base class to be used to create {@link RecordReader} instances that convert from Accumulo {@link Key}/{@link Value} pairs to the user's K/V
 +   * types.
 +   * 
 +   * Subclasses must implement {@link #nextKeyValue()} and use it to update the following variables:
 +   * <ul>
 +   * <li>K {@link #currentK}</li>
 +   * <li>V {@link #currentV}</li>
 +   * <li>Key {@link #currentKey} (used for progress reporting)</li>
 +   * <li>int {@link #numKeysRead} (used for progress reporting)</li>
 +   * </ul>
 +   */
 +  protected abstract static class RecordReaderBase<K,V> extends RecordReader<K,V> {
 +    protected long numKeysRead;
 +    protected Iterator<Entry<Key,Value>> scannerIterator;
 +    protected RangeInputSplit split;
 +    
 +    /**
 +     * Apply the configured iterators from the configuration to the scanner.
 +     * 
 +     * @param context
 +     *          the Hadoop context for the configured job
 +     * @param scanner
 +     *          the scanner to configure
 +     */
-     protected void setupIterators(TaskAttemptContext context, Scanner scanner) {
-       List<IteratorSetting> iterators = getIterators(context);
++    protected void setupIterators(List<IteratorSetting> iterators, Scanner scanner) {
 +      for (IteratorSetting iterator : iterators) {
 +        scanner.addScanIterator(iterator);
 +      }
 +    }
 +    
 +    /**
 +     * Initialize a scanner over the given input split using this task attempt configuration.
 +     */
 +    @Override
 +    public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
 +      Scanner scanner;
 +      split = (RangeInputSplit) inSplit;
-       log.debug("Initializing input split: " + split.range);
-       Instance instance = getInstance(attempt);
-       String principal = getPrincipal(attempt);
-       String tokenClass = getTokenClass(attempt);
-       byte[] token = getToken(attempt);
-       Authorizations authorizations = getScanAuthorizations(attempt);
++      log.debug("Initializing input split: " + split.getRange());
++
++      Instance instance = split.getInstance();
++      if (null == instance) {
++        instance = getInstance(attempt);
++      }
++
++      String principal = split.getPrincipal();
++      if (null == principal) {
++        principal = getPrincipal(attempt);
++      }
++
++      AuthenticationToken token = split.getToken();
++      if (null == token) {
++        String tokenClass = getTokenClass(attempt);
++        byte[] tokenBytes = getToken(attempt);
++        try {
++          token = CredentialHelper.extractToken(tokenClass, tokenBytes);
++        } catch (AccumuloSecurityException e) {
++          throw new IOException(e);
++        }
++      }
++
++      Authorizations authorizations = split.getAuths();
++      if (null == authorizations) {
++        authorizations = getScanAuthorizations(attempt);
++      }
++
++      String table = split.getTable();
++      if (null == table) {
++        table = getInputTableName(attempt);
++      }
++      
++      Boolean isOffline = split.isOffline();
++      if (null == isOffline) {
++        isOffline = isOfflineScan(attempt);
++      }
++
++      Boolean isIsolated = split.isIsolatedScan();
++      if (null == isIsolated) {
++        isIsolated = isIsolated(attempt);
++      }
++
++      Boolean usesLocalIterators = split.usesLocalIterators();
++      if (null == usesLocalIterators) {
++        usesLocalIterators = usesLocalIterators(attempt);
++      }
++      
++      List<IteratorSetting> iterators = split.getIterators();
++      if (null == iterators) {
++        iterators = getIterators(attempt);
++      }
++      
++      Set<Pair<Text,Text>> columns = split.getFetchedColumns();
++      if (null == columns) {
++        columns = getFetchedColumns(attempt);
++      }
 +      
 +      try {
 +        log.debug("Creating connector with user: " + principal);
-         Connector conn = instance.getConnector(principal, CredentialHelper.extractToken(tokenClass, token));
-         log.debug("Creating scanner for table: " + getInputTableName(attempt));
++        Connector conn = instance.getConnector(principal, token);
++        log.debug("Creating scanner for table: " + table);
 +        log.debug("Authorizations are: " + authorizations);
 +        if (isOfflineScan(attempt)) {
-           scanner = new OfflineScanner(instance, new TCredentials(principal, tokenClass, ByteBuffer.wrap(token), instance.getInstanceID()), Tables.getTableId(
-               instance, getInputTableName(attempt)), authorizations);
++          String tokenClass = token.getClass().getCanonicalName();
++          ByteBuffer tokenBuffer = ByteBuffer.wrap(CredentialHelper.toBytes(token));
++          scanner = new OfflineScanner(instance, new TCredentials(principal, tokenClass, tokenBuffer, instance.getInstanceID()), Tables.getTableId(
++              instance, table), authorizations);
 +        } else {
-           scanner = conn.createScanner(getInputTableName(attempt), authorizations);
++          scanner = conn.createScanner(table, authorizations);
 +        }
-         if (isIsolated(attempt)) {
++        if (isIsolated) {
 +          log.info("Creating isolated scanner");
 +          scanner = new IsolatedScanner(scanner);
 +        }
-         if (usesLocalIterators(attempt)) {
++        if (usesLocalIterators) {
 +          log.info("Using local iterators");
 +          scanner = new ClientSideIteratorScanner(scanner);
 +        }
-         setupIterators(attempt, scanner);
++        setupIterators(iterators, scanner);
 +      } catch (Exception e) {
 +        throw new IOException(e);
 +      }
 +      
 +      // setup a scanner within the bounds of this split
-       for (Pair<Text,Text> c : getFetchedColumns(attempt)) {
++      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.range);
++      scanner.setRange(split.getRange());
 +      
 +      numKeysRead = 0;
 +      
 +      // do this last after setting all scanner options
 +      scannerIterator = scanner.iterator();
 +    }
 +    
 +    @Override
 +    public void close() {}
 +    
 +    @Override
 +    public float getProgress() throws IOException {
 +      if (numKeysRead > 0 && currentKey == null)
 +        return 1.0f;
 +      return split.getProgress(currentKey);
 +    }
 +    
 +    protected K currentK = null;
 +    protected V currentV = null;
 +    protected Key currentKey = null;
 +    protected Value currentValue = null;
 +    
 +    @Override
 +    public K getCurrentKey() throws IOException, InterruptedException {
 +      return currentK;
 +    }
 +    
 +    @Override
 +    public V getCurrentValue() throws IOException, InterruptedException {
 +      return currentV;
 +    }
 +  }
 +  
 +  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext context, String tableName, List<Range> ranges) throws TableNotFoundException,
 +      AccumuloException, AccumuloSecurityException {
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +    
 +    Instance instance = getInstance(context);
 +    Connector conn = instance.getConnector(getPrincipal(context), CredentialHelper.extractToken(getTokenClass(context), getToken(context)));
 +    String tableId = Tables.getTableId(instance, tableName);
 +    
 +    if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
 +      Tables.clearCache(instance);
 +      if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
 +        throw new AccumuloException("Table is online " + tableName + "(" + tableId + ") cannot scan table in offline mode ");
 +      }
 +    }
 +    
 +    for (Range range : ranges) {
 +      Text startRow;
 +      
 +      if (range.getStartKey() != null)
 +        startRow = range.getStartKey().getRow();
 +      else
 +        startRow = new Text();
 +      
 +      Range metadataRange = new Range(new KeyExtent(new Text(tableId), startRow, null).getMetadataEntry(), true, null, false);
 +      Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
 +      Constants.METADATA_PREV_ROW_COLUMN.fetch(scanner);
 +      scanner.fetchColumnFamily(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY);
 +      scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
 +      scanner.fetchColumnFamily(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY);
 +      scanner.setRange(metadataRange);
 +      
 +      RowIterator rowIter = new RowIterator(scanner);
 +      
 +      KeyExtent lastExtent = null;
 +      
 +      while (rowIter.hasNext()) {
 +        Iterator<Entry<Key,Value>> row = rowIter.next();
 +        String last = "";
 +        KeyExtent extent = null;
 +        String location = null;
 +        
 +        while (row.hasNext()) {
 +          Entry<Key,Value> entry = row.next();
 +          Key key = entry.getKey();
 +          
 +          if (key.getColumnFamily().equals(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY)) {
 +            last = entry.getValue().toString();
 +          }
 +          
 +          if (key.getColumnFamily().equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY)
 +              || key.getColumnFamily().equals(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY)) {
 +            location = entry.getValue().toString();
 +          }
 +          
 +          if (Constants.METADATA_PREV_ROW_COLUMN.hasColumns(key)) {
 +            extent = new KeyExtent(key.getRow(), entry.getValue());
 +          }
 +          
 +        }
 +        
 +        if (location != null)
 +          return null;
 +        
 +        if (!extent.getTableId().toString().equals(tableId)) {
 +          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
 +        }
 +        
 +        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
 +          throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
 +        }
 +        
 +        Map<KeyExtent,List<Range>> tabletRanges = binnedRanges.get(last);
 +        if (tabletRanges == null) {
 +          tabletRanges = new HashMap<KeyExtent,List<Range>>();
 +          binnedRanges.put(last, tabletRanges);
 +        }
 +        
 +        List<Range> rangeList = tabletRanges.get(extent);
 +        if (rangeList == null) {
 +          rangeList = new ArrayList<Range>();
 +          tabletRanges.put(extent, rangeList);
 +        }
 +        
 +        rangeList.add(range);
 +        
 +        if (extent.getEndRow() == null || range.afterEndKey(new Key(extent.getEndRow()).followingKey(PartialKey.ROW))) {
 +          break;
 +        }
 +        
 +        lastExtent = extent;
 +      }
 +      
 +    }
 +    
 +    return binnedRanges;
 +  }
 +  
 +  /**
 +   * Read the metadata table to get tablets and match up ranges to them.
 +   */
 +  @Override
 +  public List<InputSplit> getSplits(JobContext context) throws IOException {
 +    log.setLevel(getLogLevel(context));
 +    validateOptions(context);
 +    
 +    String tableName = getInputTableName(context);
 +    boolean autoAdjust = getAutoAdjustRanges(context);
 +    List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(context)) : getRanges(context);
 +    
 +    if (ranges.isEmpty()) {
 +      ranges = new ArrayList<Range>(1);
 +      ranges.add(new Range());
 +    }
 +    
 +    // get the metadata information for these ranges
 +    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +    TabletLocator tl;
 +    try {
 +      if (isOfflineScan(context)) {
 +        binnedRanges = binOfflineTable(context, tableName, ranges);
 +        while (binnedRanges == null) {
 +          // Some tablets were still online, try again
 +          UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
 +          binnedRanges = binOfflineTable(context, tableName, ranges);
 +        }
 +      } else {
 +        Instance instance = getInstance(context);
 +        String tableId = null;
 +        tl = getTabletLocator(context);
 +        // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
 +        tl.invalidateCache();
 +        while (!tl.binRanges(ranges, binnedRanges,
 +            new TCredentials(getPrincipal(context), getTokenClass(context), ByteBuffer.wrap(getToken(context)), getInstance(context).getInstanceID()))
 +            .isEmpty()) {
 +          if (!(instance instanceof MockInstance)) {
 +            if (tableId == null)
 +              tableId = Tables.getTableId(instance, tableName);
 +            if (!Tables.exists(instance, tableId))
 +              throw new TableDeletedException(tableId);
 +            if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
 +              throw new TableOfflineException(instance, tableId);
 +          }
 +          binnedRanges.clear();
 +          log.warn("Unable to locate bins for specified ranges. Retrying.");
 +          UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
 +          tl.invalidateCache();
 +        }
 +      }
 +    } catch (Exception e) {
 +      throw new IOException(e);
 +    }
 +    
 +    ArrayList<InputSplit> splits = new ArrayList<InputSplit>(ranges.size());
 +    HashMap<Range,ArrayList<String>> splitsToAdd = null;
 +    
 +    if (!autoAdjust)
 +      splitsToAdd = new HashMap<Range,ArrayList<String>>();
 +    
 +    HashMap<String,String> hostNameCache = new HashMap<String,String>();
 +    
 +    for (Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
 +      String ip = tserverBin.getKey().split(":", 2)[0];
 +      String location = hostNameCache.get(ip);
 +      if (location == null) {
 +        InetAddress inetAddress = InetAddress.getByName(ip);
 +        location = inetAddress.getHostName();
 +        hostNameCache.put(ip, location);
 +      }
 +      
 +      for (Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
 +        Range ke = extentRanges.getKey().toDataRange();
 +        for (Range r : extentRanges.getValue()) {
 +          if (autoAdjust) {
 +            // divide ranges into smaller ranges, based on the tablets
-             splits.add(new RangeInputSplit(tableName, ke.clip(r), new String[] {location}));
++            splits.add(new RangeInputSplit(ke.clip(r), new String[] {location}));
 +          } else {
 +            // don't divide ranges
 +            ArrayList<String> locations = splitsToAdd.get(r);
 +            if (locations == null)
 +              locations = new ArrayList<String>(1);
 +            locations.add(location);
 +            splitsToAdd.put(r, locations);
 +          }
 +        }
 +      }
 +    }
 +    
 +    if (!autoAdjust)
 +      for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
-         splits.add(new RangeInputSplit(tableName, entry.getKey(), entry.getValue().toArray(new String[0])));
++        splits.add(new RangeInputSplit(entry.getKey(), entry.getValue().toArray(new String[0])));
 +    return splits;
 +  }
-   
-   /**
-    * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
-    */
-   public static class RangeInputSplit extends InputSplit implements Writable {
-     private Range range;
-     private String[] locations;
-     
-     public RangeInputSplit() {
-       range = new Range();
-       locations = new String[0];
-     }
-     
-     public RangeInputSplit(RangeInputSplit split) throws IOException {
-       this.setRange(split.getRange());
-       this.setLocations(split.getLocations());
-     }
-     
-     protected RangeInputSplit(String table, Range range, String[] locations) {
-       this.range = range;
-       this.locations = locations;
-     }
-     
-     public Range getRange() {
-       return range;
-     }
-     
-     public void setRange(Range range) {
-       this.range = range;
-     }
-     
-     private static byte[] extractBytes(ByteSequence seq, int numBytes) {
-       byte[] bytes = new byte[numBytes + 1];
-       bytes[0] = 0;
-       for (int i = 0; i < numBytes; i++) {
-         if (i >= seq.length())
-           bytes[i + 1] = 0;
-         else
-           bytes[i + 1] = seq.byteAt(i);
-       }
-       return bytes;
-     }
-     
-     public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
-       int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
-       BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
-       BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
-       BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
-       return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
-     }
-     
-     public float getProgress(Key currentKey) {
-       if (currentKey == null)
-         return 0f;
-       if (range.getStartKey() != null && range.getEndKey() != null) {
-         if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW)) {
-           // just look at the row progress
-           return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
-         } else if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW_COLFAM)) {
-           // just look at the column family progress
-           return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
-         } else if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL)) {
-           // just look at the column qualifier progress
-           return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
-         }
-       }
-       // if we can't figure it out, then claim no progress
-       return 0f;
-     }
-     
-     /**
-      * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
-      */
-     @Override
-     public long getLength() throws IOException {
-       Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
-       Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
-       int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
-       long diff = 0;
-       
-       byte[] start = startRow.getBytes();
-       byte[] stop = stopRow.getBytes();
-       for (int i = 0; i < maxCommon; ++i) {
-         diff |= 0xff & (start[i] ^ stop[i]);
-         diff <<= Byte.SIZE;
-       }
-       
-       if (startRow.getLength() != stopRow.getLength())
-         diff |= 0xff;
-       
-       return diff + 1;
-     }
-     
-     @Override
-     public String[] getLocations() throws IOException {
-       return locations;
-     }
-     
-     public void setLocations(String[] locations) {
-       this.locations = locations;
-     }
-     
-     @Override
-     public void readFields(DataInput in) throws IOException {
-       range.readFields(in);
-       int numLocs = in.readInt();
-       locations = new String[numLocs];
-       for (int i = 0; i < numLocs; ++i)
-         locations[i] = in.readUTF();
-     }
-     
-     @Override
-     public void write(DataOutput out) throws IOException {
-       range.write(out);
-       out.writeInt(locations.length);
-       for (int i = 0; i < locations.length; ++i)
-         out.writeUTF(locations[i]);
-     }
-   }
-   
++
 +  // ----------------------------------------------------------------------------------------------------
 +  // Everything below this line is deprecated and should go away in future versions
 +  // ----------------------------------------------------------------------------------------------------
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #setScanIsolation(Job, boolean)} instead.
 +   */
 +  @Deprecated
 +  public static void setIsolated(Configuration conf, boolean enable) {
 +    InputConfigurator.setScanIsolation(CLASS, conf, enable);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #setLocalIterators(Job, boolean)} instead.
 +   */
 +  @Deprecated
 +  public static void setLocalIterators(Configuration conf, boolean enable) {
 +    InputConfigurator.setLocalIterators(CLASS, conf, enable);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #setConnectorInfo(Job, String, AuthenticationToken)}, {@link #setInputTableName(Job, String)}, and
 +   *             {@link #setScanAuthorizations(Job, Authorizations)} instead.
 +   */
 +  @Deprecated
 +  public static void setInputInfo(Configuration conf, String user, byte[] passwd, String table, Authorizations auths) {
 +    try {
 +      InputConfigurator.setConnectorInfo(CLASS, conf, user, new PasswordToken(passwd));
 +    } catch (AccumuloSecurityException e) {
 +      throw new RuntimeException(e);
 +    }
 +    InputConfigurator.setInputTableName(CLASS, conf, table);
 +    InputConfigurator.setScanAuthorizations(CLASS, conf, auths);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #setZooKeeperInstance(Job, String, String)} instead.
 +   */
 +  @Deprecated
 +  public static void setZooKeeperInstance(Configuration conf, String instanceName, String zooKeepers) {
 +    InputConfigurator.setZooKeeperInstance(CLASS, conf, instanceName, zooKeepers);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #setMockInstance(Job, String)} instead.
 +   */
 +  @Deprecated
 +  public static void setMockInstance(Configuration conf, String instanceName) {
 +    InputConfigurator.setMockInstance(CLASS, conf, instanceName);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #setRanges(Job, Collection)} instead.
 +   */
 +  @Deprecated
 +  public static void setRanges(Configuration conf, Collection<Range> ranges) {
 +    InputConfigurator.setRanges(CLASS, conf, ranges);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #setAutoAdjustRanges(Job, boolean)} instead.
 +   */
 +  @Deprecated
 +  public static void disableAutoAdjustRanges(Configuration conf) {
 +    InputConfigurator.setAutoAdjustRanges(CLASS, conf, false);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #addIterator(Job, IteratorSetting)} to add the {@link VersioningIterator} instead.
 +   */
 +  @Deprecated
 +  public static void setMaxVersions(Configuration conf, int maxVersions) throws IOException {
 +    IteratorSetting vers = new IteratorSetting(1, "vers", VersioningIterator.class);
 +    try {
 +      VersioningIterator.setMaxVersions(vers, maxVersions);
 +    } catch (IllegalArgumentException e) {
 +      throw new IOException(e);
 +    }
 +    InputConfigurator.addIterator(CLASS, conf, vers);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #setOfflineTableScan(Job, boolean)} instead.
 +   */
 +  @Deprecated
 +  public static void setScanOffline(Configuration conf, boolean scanOff) {
 +    InputConfigurator.setOfflineTableScan(CLASS, conf, scanOff);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #fetchColumns(Job, Collection)} instead.
 +   */
 +  @Deprecated
 +  public static void fetchColumns(Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 +    InputConfigurator.fetchColumns(CLASS, conf, columnFamilyColumnQualifierPairs);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #setLogLevel(Job, Level)} instead.
 +   */
 +  @Deprecated
 +  public static void setLogLevel(Configuration conf, Level level) {
 +    InputConfigurator.setLogLevel(CLASS, conf, level);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #addIterator(Job, IteratorSetting)} instead.
 +   */
 +  @Deprecated
 +  public static void addIterator(Configuration conf, IteratorSetting cfg) {
 +    InputConfigurator.addIterator(CLASS, conf, cfg);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #isIsolated(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static boolean isIsolated(Configuration conf) {
 +    return InputConfigurator.isIsolated(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #usesLocalIterators(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static boolean usesLocalIterators(Configuration conf) {
 +    return InputConfigurator.usesLocalIterators(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getPrincipal(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static String getPrincipal(Configuration conf) {
 +    return InputConfigurator.getPrincipal(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getToken(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static byte[] getToken(Configuration conf) {
 +    return InputConfigurator.getToken(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getInputTableName(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static String getTablename(Configuration conf) {
 +    return InputConfigurator.getInputTableName(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getScanAuthorizations(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static Authorizations getAuthorizations(Configuration conf) {
 +    return InputConfigurator.getScanAuthorizations(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getInstance(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static Instance getInstance(Configuration conf) {
 +    return InputConfigurator.getInstance(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getTabletLocator(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static TabletLocator getTabletLocator(Configuration conf) throws TableNotFoundException {
 +    return InputConfigurator.getTabletLocator(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getRanges(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static List<Range> getRanges(Configuration conf) throws IOException {
 +    return InputConfigurator.getRanges(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getFetchedColumns(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static Set<Pair<Text,Text>> getFetchedColumns(Configuration conf) {
 +    return InputConfigurator.getFetchedColumns(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getAutoAdjustRanges(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static boolean getAutoAdjustRanges(Configuration conf) {
 +    return InputConfigurator.getAutoAdjustRanges(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getLogLevel(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static Level getLogLevel(Configuration conf) {
 +    return InputConfigurator.getLogLevel(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #validateOptions(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static void validateOptions(Configuration conf) throws IOException {
 +    InputConfigurator.validateOptions(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #addIterator(Job, IteratorSetting)} to add the {@link VersioningIterator} instead.
 +   */
 +  @Deprecated
 +  protected static int getMaxVersions(Configuration conf) {
 +    // This is so convoluted, because the only reason to get the number of maxVersions is to construct the same type of IteratorSetting object we have to
 +    // deconstruct to get at this option in the first place, but to preserve correct behavior, this appears necessary.
 +    List<IteratorSetting> iteratorSettings = InputConfigurator.getIterators(CLASS, conf);
 +    for (IteratorSetting setting : iteratorSettings) {
 +      if ("vers".equals(setting.getName()) && 1 == setting.getPriority() && VersioningIterator.class.getName().equals(setting.getIteratorClass())) {
 +        if (setting.getOptions().containsKey("maxVersions"))
 +          return Integer.parseInt(setting.getOptions().get("maxVersions"));
 +        else
 +          return -1;
 +      }
 +    }
 +    return -1;
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #isOfflineScan(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static boolean isOfflineScan(Configuration conf) {
 +    return InputConfigurator.isOfflineScan(CLASS, conf);
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getIterators(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static List<AccumuloIterator> getIterators(Configuration conf) {
 +    List<IteratorSetting> iteratorSettings = InputConfigurator.getIterators(CLASS, conf);
 +    List<AccumuloIterator> deprecatedIterators = new ArrayList<AccumuloIterator>(iteratorSettings.size());
 +    for (IteratorSetting setting : iteratorSettings) {
 +      AccumuloIterator deprecatedIter = new AccumuloIterator(new String(setting.getPriority() + AccumuloIterator.FIELD_SEP + setting.getIteratorClass()
 +          + AccumuloIterator.FIELD_SEP + setting.getName()));
 +      deprecatedIterators.add(deprecatedIter);
 +    }
 +    return deprecatedIterators;
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link #getIterators(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static List<AccumuloIteratorOption> getIteratorOptions(Configuration conf) {
 +    List<IteratorSetting> iteratorSettings = InputConfigurator.getIterators(CLASS, conf);
 +    List<AccumuloIteratorOption> deprecatedIteratorOptions = new ArrayList<AccumuloIteratorOption>(iteratorSettings.size());
 +    for (IteratorSetting setting : iteratorSettings) {
 +      for (Entry<String,String> opt : setting.getOptions().entrySet()) {
 +        String deprecatedOption;
 +        try {
 +          deprecatedOption = new String(setting.getName() + AccumuloIteratorOption.FIELD_SEP + URLEncoder.encode(opt.getKey(), "UTF-8")
 +              + AccumuloIteratorOption.FIELD_SEP + URLEncoder.encode(opt.getValue(), "UTF-8"));
 +        } catch (UnsupportedEncodingException e) {
 +          throw new RuntimeException(e);
 +        }
 +        deprecatedIteratorOptions.add(new AccumuloIteratorOption(deprecatedOption));
 +      }
 +    }
 +    return deprecatedIteratorOptions;
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link IteratorSetting} instead.
 +   */
 +  @Deprecated
 +  static class AccumuloIterator {
 +    
 +    private static final String FIELD_SEP = ":";
 +    
 +    private int priority;
 +    private String iteratorClass;
 +    private String iteratorName;
 +    
 +    public AccumuloIterator(int priority, String iteratorClass, String iteratorName) {
 +      this.priority = priority;
 +      this.iteratorClass = iteratorClass;
 +      this.iteratorName = iteratorName;
 +    }
 +    
 +    // Parses out a setting given an string supplied from an earlier toString() call
 +    public AccumuloIterator(String iteratorSetting) {
 +      // Parse the string to expand the iterator
 +      StringTokenizer tokenizer = new StringTokenizer(iteratorSetting, FIELD_SEP);
 +      priority = Integer.parseInt(tokenizer.nextToken());
 +      iteratorClass = tokenizer.nextToken();
 +      iteratorName = tokenizer.nextToken();
 +    }
 +    
 +    public int getPriority() {
 +      return priority;
 +    }
 +    
 +    public String getIteratorClass() {
 +      return iteratorClass;
 +    }
 +    
 +    public String getIteratorName() {
 +      return iteratorName;
 +    }
 +    
 +    @Override
 +    public String toString() {
 +      return new String(priority + FIELD_SEP + iteratorClass + FIELD_SEP + iteratorName);
 +    }
 +    
 +  }
 +  
 +  /**
 +   * @deprecated since 1.5.0; Use {@link IteratorSetting} instead.
 +   */
 +  @Deprecated
 +  static class AccumuloIteratorOption {
 +    private static final String FIELD_SEP = ":";
 +    
 +    private String iteratorName;
 +    private String key;
 +    private String value;
 +    
 +    public AccumuloIteratorOption(String iteratorName, String key, String value) {
 +      this.iteratorName = iteratorName;
 +      this.key = key;
 +      this.value = value;
 +    }
 +    
 +    // Parses out an option given a string supplied from an earlier toString() call
 +    public AccumuloIteratorOption(String iteratorOption) {
 +      StringTokenizer tokenizer = new StringTokenizer(iteratorOption, FIELD_SEP);
 +      this.iteratorName = tokenizer.nextToken();
 +      try {
 +        this.key = URLDecoder.decode(tokenizer.nextToken(), "UTF-8");
 +        this.value = URLDecoder.decode(tokenizer.nextToken(), "UTF-8");
 +      } catch (UnsupportedEncodingException e) {
 +        throw new RuntimeException(e);
 +      }
 +    }
 +    
 +    public String getIteratorName() {
 +      return iteratorName;
 +    }
 +    
 +    public String getKey() {
 +      return key;
 +    }
 +    
 +    public String getValue() {
 +      return value;
 +    }
 +    
 +    @Override
 +    public String toString() {
 +      try {
 +        return new String(iteratorName + FIELD_SEP + URLEncoder.encode(key, "UTF-8") + FIELD_SEP + URLEncoder.encode(value, "UTF-8"));
 +      } catch (UnsupportedEncodingException e) {
 +        throw new RuntimeException(e);
 +      }
 +    }
 +    
 +  }
 +  
 +  // use reflection to pull the Configuration out of the JobContext for Hadoop 1 and Hadoop 2 compatibility
 +  static Configuration getConfiguration(JobContext context) {
 +    try {
 +      Class<?> c = InputFormatBase.class.getClassLoader().loadClass("org.apache.hadoop.mapreduce.JobContext");
 +      Method m = c.getMethod("getConfiguration");
 +      Object o = m.invoke(context, new Object[0]);
 +      return (Configuration) o;
 +    } catch (Exception e) {
 +      throw new RuntimeException(e);
 +    }
 +  }
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/16a2e0f0/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index 0000000,0000000..d613836
new file mode 100644
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@@ -1,0 -1,0 +1,503 @@@
++/*
++ * Licensed to the Apache Software Foundation (ASF) under one or more
++ * contributor license agreements.  See the NOTICE file distributed with
++ * this work for additional information regarding copyright ownership.
++ * The ASF licenses this file to You under the Apache License, Version 2.0
++ * (the "License"); you may not use this file except in compliance with
++ * the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++package org.apache.accumulo.core.client.mapreduce;
++
++import java.io.DataInput;
++import java.io.DataOutput;
++import java.io.IOException;
++import java.math.BigInteger;
++import java.nio.charset.Charset;
++import java.util.ArrayList;
++import java.util.List;
++import java.util.Set;
++
++import org.apache.accumulo.core.client.AccumuloSecurityException;
++import org.apache.accumulo.core.client.Instance;
++import org.apache.accumulo.core.client.IteratorSetting;
++import org.apache.accumulo.core.client.ZooKeeperInstance;
++import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
++import org.apache.accumulo.core.client.mock.MockInstance;
++import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
++import org.apache.accumulo.core.data.ByteSequence;
++import org.apache.accumulo.core.data.Key;
++import org.apache.accumulo.core.data.PartialKey;
++import org.apache.accumulo.core.data.Range;
++import org.apache.accumulo.core.security.Authorizations;
++import org.apache.accumulo.core.security.CredentialHelper;
++import org.apache.accumulo.core.util.Pair;
++import org.apache.commons.codec.binary.Base64;
++import org.apache.hadoop.io.Text;
++import org.apache.hadoop.io.Writable;
++import org.apache.hadoop.mapreduce.InputSplit;
++import org.apache.hadoop.util.StringUtils;
++import org.apache.log4j.Level;
++
++/**
++ * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
++ */
++public class RangeInputSplit extends InputSplit implements Writable {
++  private Range range;
++  private String[] locations;
++  private String table, instanceName, zooKeepers, principal;
++  private String rowRegex, colfamRegex, colqualRegex, valueRegex;
++  private AuthenticationToken token;
++  private Boolean offline, mockInstance, isolatedScan, localIterators;
++  private Integer maxVersions;
++  private Authorizations auths;
++  private Set<Pair<Text,Text>> fetchedColumns;
++  private List<IteratorSetting> iterators;
++  private Level level;
++
++  public RangeInputSplit() {
++    range = new Range();
++    locations = new String[0];
++  }
++
++  public RangeInputSplit(Range range, String[] locations) {
++    this.range = range;
++    this.locations = locations;
++  }
++
++  public Range getRange() {
++    return range;
++  }
++
++  private static byte[] extractBytes(ByteSequence seq, int numBytes) {
++    byte[] bytes = new byte[numBytes + 1];
++    bytes[0] = 0;
++    for (int i = 0; i < numBytes; i++) {
++      if (i >= seq.length())
++        bytes[i + 1] = 0;
++      else
++        bytes[i + 1] = seq.byteAt(i);
++    }
++    return bytes;
++  }
++
++  public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
++    int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
++    BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
++    BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
++    BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
++    return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
++  }
++
++  public float getProgress(Key currentKey) {
++    if (currentKey == null)
++      return 0f;
++    if (range.getStartKey() != null && range.getEndKey() != null) {
++      if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
++        // just look at the row progress
++        return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
++      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
++        // just look at the column family progress
++        return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
++      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL) != 0) {
++        // just look at the column qualifier progress
++        return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
++      }
++    }
++    // if we can't figure it out, then claim no progress
++    return 0f;
++  }
++
++  /**
++   * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
++   */
++  public long getLength() throws IOException {
++    Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
++    Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
++    int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
++    long diff = 0;
++
++    byte[] start = startRow.getBytes();
++    byte[] stop = stopRow.getBytes();
++    for (int i = 0; i < maxCommon; ++i) {
++      diff |= 0xff & (start[i] ^ stop[i]);
++      diff <<= Byte.SIZE;
++    }
++
++    if (startRow.getLength() != stopRow.getLength())
++      diff |= 0xff;
++
++    return diff + 1;
++  }
++
++  public String[] getLocations() throws IOException {
++    return locations;
++  }
++
++  public void readFields(DataInput in) throws IOException {
++    range.readFields(in);
++    int numLocs = in.readInt();
++    locations = new String[numLocs];
++    for (int i = 0; i < numLocs; ++i)
++      locations[i] = in.readUTF();
++    
++    if (in.readBoolean()) {
++      isolatedScan = in.readBoolean();
++    }
++    
++    if (in.readBoolean()) {
++      offline = in.readBoolean();
++    }
++    
++    if (in.readBoolean()) {
++      localIterators = in.readBoolean();
++    }
++    
++    if (in.readBoolean()) {
++      mockInstance = in.readBoolean();
++    }
++    
++    if (in.readBoolean()) {
++      maxVersions = in.readInt();
++    }
++    
++    if (in.readBoolean()) {
++      rowRegex = in.readUTF();
++    }
++    
++    if (in.readBoolean()) {
++      colfamRegex = in.readUTF();
++    }
++    
++    if (in.readBoolean()) {
++      colqualRegex = in.readUTF();
++    }
++    
++    if (in.readBoolean()) {
++      valueRegex = in.readUTF();
++    }
++    
++    if (in.readBoolean()) {
++      int numColumns = in.readInt();
++      List<String> columns = new ArrayList<String>(numColumns);
++      for (int i = 0; i < numColumns; i++) {
++        columns.set(i, in.readUTF());
++      }
++      
++      fetchedColumns = InputConfigurator.deserializeFetchedColumns(columns);
++    }
++    
++    if (in.readBoolean()) {
++      auths = new Authorizations(StringUtils.split(in.readUTF()));
++    }
++    
++    if (in.readBoolean()) {
++      principal = in.readUTF();
++    }
++    
++    if (in.readBoolean()) {
++      String tokenClass = in.readUTF();
++      byte[] base64TokenBytes = in.readUTF().getBytes(Charset.forName("UTF-8"));
++      byte[] tokenBytes = Base64.decodeBase64(base64TokenBytes);
++      
++      try {
++        token = CredentialHelper.extractToken(tokenClass, tokenBytes);
++      } catch (AccumuloSecurityException e) {
++        throw new IOException(e);
++      }
++    }
++    
++    if (in.readBoolean()) {
++      instanceName = in.readUTF();
++    }
++    
++    if (in.readBoolean()) {
++      zooKeepers = in.readUTF();
++    }
++    
++    if (in.readBoolean()) {
++      level = Level.toLevel(in.readInt());
++    }
++  }
++
++  public void write(DataOutput out) throws IOException {
++    range.write(out);
++    out.writeInt(locations.length);
++    for (int i = 0; i < locations.length; ++i)
++      out.writeUTF(locations[i]);
++    
++    out.writeBoolean(null != isolatedScan);
++    if (null != isolatedScan) {
++      out.writeBoolean(isolatedScan);
++    }
++    
++    out.writeBoolean(null != offline);
++    if (null != offline) {
++      out.writeBoolean(offline);
++    }
++    
++    out.writeBoolean(null != localIterators);
++    if (null != localIterators) {
++      out.writeBoolean(localIterators);
++    }
++    
++    out.writeBoolean(null != mockInstance);
++    if (null != mockInstance) {
++      out.writeBoolean(mockInstance);
++    }
++    
++    out.writeBoolean(null != maxVersions);
++    if (null != maxVersions) {
++      out.writeInt(getMaxVersions());
++    }
++    
++    out.writeBoolean(null != rowRegex);
++    if (null != rowRegex) {
++      out.writeUTF(rowRegex);
++    }
++    
++    out.writeBoolean(null != colfamRegex);
++    if (null != colfamRegex) {
++      out.writeUTF(colfamRegex);
++    }
++    
++    out.writeBoolean(null != colqualRegex);
++    if (null != colqualRegex) {
++      out.writeUTF(colqualRegex);
++    }
++    
++    out.writeBoolean(null != valueRegex);
++    if (null != valueRegex) {
++      out.writeUTF(valueRegex);
++    }
++    
++    out.writeBoolean(null != fetchedColumns);
++    if (null != fetchedColumns) {
++      String[] cols = InputConfigurator.serializeColumns(fetchedColumns);
++      out.writeInt(cols.length);
++      for (String col : cols) {
++        out.writeUTF(col);
++      }
++    }
++    
++    out.writeBoolean(null != auths);
++    if (null != auths) {
++      out.writeUTF(auths.serialize());
++    }
++    
++    out.writeBoolean(null != principal);
++    if (null != principal) {
++      out.writeUTF(principal);
++    }
++    
++    out.writeBoolean(null != token);
++    if (null != token) {
++      out.writeUTF(token.getClass().getCanonicalName());
++      try {
++        out.writeUTF(CredentialHelper.tokenAsBase64(token));
++      } catch (AccumuloSecurityException e) {
++        throw new IOException(e);
++      }
++    }
++    
++    out.writeBoolean(null != instanceName);
++    if (null != instanceName) {
++      out.writeUTF(instanceName);
++    }
++    
++    out.writeBoolean(null != zooKeepers);
++    if (null != zooKeepers) {
++      out.writeUTF(zooKeepers);
++    }
++    
++    out.writeBoolean(null != level);
++    if (null != level) {
++      out.writeInt(level.toInt());
++    }
++  }
++
++  @Override
++  public String toString() {
++    StringBuilder sb = new StringBuilder(128);
++    sb.append("Range: ").append(range);
++    sb.append(" Locations: ").append(locations);
++    sb.append(" Table: ").append(table);
++    // TODO finish building of string
++    return sb.toString();
++  }
++
++  public String getTable() {
++    return table;
++  }
++
++  public void setTable(String table) {
++    this.table = table;
++  }
++  
++  public Instance getInstance() {
++    if (null == instanceName) {
++      return null;
++    }
++    
++    if (isMockInstance()) {  
++      return new MockInstance(getInstanceName());
++    }
++    
++    if (null == zooKeepers) {
++      return null;
++    }
++    
++    return new ZooKeeperInstance(getInstanceName(), getZooKeepers());
++  }
++
++  public String getInstanceName() {
++    return instanceName;
++  }
++
++  public void setInstanceName(String instanceName) {
++    this.instanceName = instanceName;
++  }
++
++  public String getZooKeepers() {
++    return zooKeepers;
++  }
++
++  public void setZooKeepers(String zooKeepers) {
++    this.zooKeepers = zooKeepers;
++  }
++
++  public String getPrincipal() {
++    return principal;
++  }
++
++  public void setPrincipal(String principal) {
++    this.principal = principal;
++  }
++  
++  public AuthenticationToken getToken() {
++    return token;
++  }
++  
++  public void setToken(AuthenticationToken token) {
++    this.token = token;;
++  }
++
++  public Boolean isOffline() {
++    return offline;
++  }
++
++  public void setOffline(Boolean offline) {
++    this.offline = offline;
++  }
++
++  public void setLocations(String[] locations) {
++    this.locations = locations;
++  }
++
++  public String getRowRegex() {
++    return rowRegex;
++  }
++
++  public void setRowRegex(String rowRegex) {
++    this.rowRegex = rowRegex;
++  }
++
++  public String getColfamRegex() {
++    return colfamRegex;
++  }
++
++  public void setColfamRegex(String colfamRegex) {
++    this.colfamRegex = colfamRegex;
++  }
++
++  public String getColqualRegex() {
++    return colqualRegex;
++  }
++
++  public void setColqualRegex(String colqualRegex) {
++    this.colqualRegex = colqualRegex;
++  }
++
++  public String getValueRegex() {
++    return valueRegex;
++  }
++
++  public void setValueRegex(String valueRegex) {
++    this.valueRegex = valueRegex;
++  }
++
++  public Boolean isMockInstance() {
++    return mockInstance;
++  }
++
++  public void setMockInstance(Boolean mockInstance) {
++    this.mockInstance = mockInstance;
++  }
++
++  public Boolean isIsolatedScan() {
++    return isolatedScan;
++  }
++
++  public void setIsolatedScan(Boolean isolatedScan) {
++    this.isolatedScan = isolatedScan;
++  }
++
++  public Integer getMaxVersions() {
++    return maxVersions;
++  }
++
++  public void setMaxVersions(Integer maxVersions) {
++    this.maxVersions = maxVersions;
++  }
++
++  public Authorizations getAuths() {
++    return auths;
++  }
++
++  public void setAuths(Authorizations auths) {
++    this.auths = auths;
++  }
++
++  public void setRange(Range range) {
++    this.range = range;
++  }
++
++  public Boolean usesLocalIterators() {
++    return localIterators;
++  }
++
++  public void setUsesLocalIterators(Boolean localIterators) {
++    this.localIterators = localIterators;
++  }
++
++  public Set<Pair<Text,Text>> getFetchedColumns() {
++    return fetchedColumns;
++  }
++
++  public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
++    this.fetchedColumns = fetchedColumns;
++  }
++
++  public List<IteratorSetting> getIterators() {
++    return iterators;
++  }
++
++  public void setIterators(List<IteratorSetting> iterators) {
++    this.iterators = iterators;
++  }
++
++  public Level getLogLevel() {
++    return level;
++  }
++  
++  public void setLogLevel(Level level) {
++    this.level = level;
++  }
++}


[12/19] git commit: ACCUMULO-1854 Get the mapred package code working with the new approach as well.

Posted by el...@apache.org.
ACCUMULO-1854 Get the mapred package code working with the new approach as well.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/03b065d2
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/03b065d2
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/03b065d2

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 03b065d2d42945f4edf50d2510a15f523accd0b5
Parents: ef64992
Author: Josh Elser <el...@apache.org>
Authored: Fri Nov 22 12:37:21 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Nov 22 12:37:21 2013 -0500

----------------------------------------------------------------------
 .../core/client/mapred/InputFormatBase.java     | 132 +++++-
 .../core/client/mapred/RangeInputSplit.java     | 416 +++++++++++++++++++
 .../core/client/mapreduce/InputFormatBase.java  |  19 +-
 3 files changed, 533 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/03b065d2/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
index 8d3d710..0be4706 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
@@ -45,7 +45,7 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.impl.OfflineScanner;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
+import org.apache.accumulo.core.client.mapred.RangeInputSplit;
 import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -547,8 +547,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
      * @param scanner
      *          the scanner to configure
      */
-    protected void setupIterators(JobConf job, Scanner scanner) {
-      List<IteratorSetting> iterators = getIterators(job);
+    protected void setupIterators(List<IteratorSetting> iterators, Scanner scanner) {
       for (IteratorSetting iterator : iterators) {
         scanner.addScanIterator(iterator);
       }
@@ -561,38 +560,91 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
       Scanner scanner;
       split = (RangeInputSplit) inSplit;
       log.debug("Initializing input split: " + split.getRange());
-      Instance instance = getInstance(job);
-      String user = getPrincipal(job);
-      String tokenClass = getTokenClass(job);
-      byte[] password = getToken(job);
-      Authorizations authorizations = getScanAuthorizations(job);
+
+      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) {
+        String tokenClass = getTokenClass(job);
+        byte[] tokenBytes = getToken(job);
+        try {
+          token = CredentialHelper.extractToken(tokenClass, tokenBytes);
+        } catch (AccumuloSecurityException e) {
+          throw new IOException(e);
+        }
+      }
+
+      Authorizations authorizations = split.getAuths();
+      if (null == authorizations) {
+        authorizations = getScanAuthorizations(job);
+      }
+
+      String table = split.getTable();
+      if (null == table) {
+        table = getInputTableName(job);
+      }
+      
+      Boolean isOffline = split.isOffline();
+      if (null == isOffline) {
+        isOffline = isOfflineScan(job);
+      }
+
+      Boolean isIsolated = split.isIsolatedScan();
+      if (null == isIsolated) {
+        isIsolated = isIsolated(job);
+      }
+
+      Boolean usesLocalIterators = split.usesLocalIterators();
+      if (null == usesLocalIterators) {
+        usesLocalIterators = usesLocalIterators(job);
+      }
+      
+      List<IteratorSetting> iterators = split.getIterators();
+      if (null == iterators) {
+        iterators = getIterators(job);
+      }
+      
+      Set<Pair<Text,Text>> columns = split.getFetchedColumns();
+      if (null == columns) {
+        columns = getFetchedColumns(job);
+      }
       
       try {
-        log.debug("Creating connector with user: " + user);
-        Connector conn = instance.getConnector(user, CredentialHelper.extractToken(tokenClass, password));
-        log.debug("Creating scanner for table: " + getInputTableName(job));
+        log.debug("Creating connector with user: " + principal);
+        Connector conn = instance.getConnector(principal, token);
+        log.debug("Creating scanner for table: " + table);
         log.debug("Authorizations are: " + authorizations);
         if (isOfflineScan(job)) {
-          scanner = new OfflineScanner(instance, new TCredentials(user, tokenClass, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(
-              instance, getInputTableName(job)), authorizations);
+          String tokenClass = token.getClass().getCanonicalName();
+          ByteBuffer tokenBuffer = ByteBuffer.wrap(CredentialHelper.toBytes(token));
+          scanner = new OfflineScanner(instance, new TCredentials(principal, tokenClass, tokenBuffer, instance.getInstanceID()), Tables.getTableId(
+              instance, table), authorizations);
         } else {
-          scanner = conn.createScanner(getInputTableName(job), authorizations);
+          scanner = conn.createScanner(table, authorizations);
         }
-        if (isIsolated(job)) {
+        if (isIsolated) {
           log.info("Creating isolated scanner");
           scanner = new IsolatedScanner(scanner);
         }
-        if (usesLocalIterators(job)) {
+        if (usesLocalIterators) {
           log.info("Using local iterators");
           scanner = new ClientSideIteratorScanner(scanner);
         }
-        setupIterators(job, scanner);
+        setupIterators(iterators, scanner);
       } catch (Exception e) {
         throw new IOException(e);
       }
       
       // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : getFetchedColumns(job)) {
+      for (Pair<Text,Text> c : columns) {
         if (c.getSecond() != null) {
           log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
           scanner.fetchColumn(c.getFirst(), c.getSecond());
@@ -732,12 +784,33 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    */
   @Override
   public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
-    log.setLevel(getLogLevel(job));
+    Level logLevel = getLogLevel(job);
+    log.setLevel(logLevel);
+    
     validateOptions(job);
     
     String tableName = getInputTableName(job);
     boolean autoAdjust = getAutoAdjustRanges(job);
     List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(job)) : getRanges(job);
+    Instance instance = getInstance(job);
+    boolean offline = isOfflineScan(job);
+    boolean isolated = isIsolated(job);
+    boolean localIterators = usesLocalIterators(job);
+    boolean mockInstance = (null != instance && MockInstance.class.equals(instance.getClass()));
+    Set<Pair<Text,Text>> fetchedColumns = getFetchedColumns(job);
+    Authorizations auths = getScanAuthorizations(job);
+    String principal = getPrincipal(job);
+    String tokenClass = getTokenClass(job);
+    byte[] tokenBytes = getToken(job);
+    
+    AuthenticationToken token;
+    try {
+       token = CredentialHelper.extractToken(tokenClass, tokenBytes);
+    } catch (AccumuloSecurityException e) {
+      throw new IOException(e);
+    }
+    
+    List<IteratorSetting> iterators = getIterators(job);
     
     if (ranges.isEmpty()) {
       ranges = new ArrayList<Range>(1);
@@ -756,13 +829,11 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
           binnedRanges = binOfflineTable(job, tableName, ranges);
         }
       } else {
-        Instance instance = getInstance(job);
         String tableId = null;
         tl = getTabletLocator(job);
         // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
         tl.invalidateCache();
-        while (!tl.binRanges(ranges, binnedRanges,
-            new TCredentials(getPrincipal(job), getTokenClass(job), ByteBuffer.wrap(getToken(job)), getInstance(job).getInstanceID())).isEmpty()) {
+        while (!tl.binRanges(ranges, binnedRanges, new TCredentials(principal, tokenClass, ByteBuffer.wrap(tokenBytes), instance.getInstanceID())).isEmpty()) {
           if (!(instance instanceof MockInstance)) {
             if (tableId == null)
               tableId = Tables.getTableId(instance, tableName);
@@ -819,6 +890,23 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
     if (!autoAdjust)
       for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
         splits.add(new RangeInputSplit(entry.getKey(), entry.getValue().toArray(new String[0])));
+    
+    for (RangeInputSplit split : splits) {
+      split.setTable(tableName);
+      split.setOffline(offline);
+      split.setIsolatedScan(isolated);
+      split.setUsesLocalIterators(localIterators);
+      split.setMockInstance(mockInstance);
+      split.setFetchedColumns(fetchedColumns);
+      split.setPrincipal(principal);
+      split.setToken(token);
+      split.setInstanceName(instance.getInstanceName());
+      split.setZooKeepers(instance.getZooKeepers());
+      split.setAuths(auths);
+      split.setIterators(iterators);
+      split.setLogLevel(logLevel);
+    }
+    
     return splits.toArray(new InputSplit[splits.size()]);
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/03b065d2/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
new file mode 100644
index 0000000..01bf6dc
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
@@ -0,0 +1,416 @@
+/*
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.log4j.Level;
+
+/**
+ * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
+ */
+public class RangeInputSplit implements InputSplit, Writable {
+  private Range range;
+  private String[] locations;
+  private String table, instanceName, zooKeepers, principal;
+  private AuthenticationToken token;
+  private Boolean offline, mockInstance, isolatedScan, localIterators;
+  private Authorizations auths;
+  private Set<Pair<Text,Text>> fetchedColumns;
+  private List<IteratorSetting> iterators;
+  private Level level;
+
+  public RangeInputSplit() {
+    range = new Range();
+    locations = new String[0];
+  }
+
+  public RangeInputSplit(Range range, String[] locations) {
+    this.range = range;
+    this.locations = locations;
+  }
+
+  public Range getRange() {
+    return range;
+  }
+
+  private static byte[] extractBytes(ByteSequence seq, int numBytes) {
+    byte[] bytes = new byte[numBytes + 1];
+    bytes[0] = 0;
+    for (int i = 0; i < numBytes; i++) {
+      if (i >= seq.length())
+        bytes[i + 1] = 0;
+      else
+        bytes[i + 1] = seq.byteAt(i);
+    }
+    return bytes;
+  }
+
+  public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
+    int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
+    BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
+    BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
+    BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
+    return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
+  }
+
+  public float getProgress(Key currentKey) {
+    if (currentKey == null)
+      return 0f;
+    if (range.getStartKey() != null && range.getEndKey() != null) {
+      if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
+        // just look at the row progress
+        return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
+      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
+        // just look at the column family progress
+        return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
+      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL) != 0) {
+        // just look at the column qualifier progress
+        return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
+      }
+    }
+    // if we can't figure it out, then claim no progress
+    return 0f;
+  }
+
+  /**
+   * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
+   */
+  public long getLength() throws IOException {
+    Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
+    Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
+    int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
+    long diff = 0;
+
+    byte[] start = startRow.getBytes();
+    byte[] stop = stopRow.getBytes();
+    for (int i = 0; i < maxCommon; ++i) {
+      diff |= 0xff & (start[i] ^ stop[i]);
+      diff <<= Byte.SIZE;
+    }
+
+    if (startRow.getLength() != stopRow.getLength())
+      diff |= 0xff;
+
+    return diff + 1;
+  }
+
+  public String[] getLocations() throws IOException {
+    return locations;
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    range.readFields(in);
+    int numLocs = in.readInt();
+    locations = new String[numLocs];
+    for (int i = 0; i < numLocs; ++i)
+      locations[i] = in.readUTF();
+    
+    if (in.readBoolean()) {
+      isolatedScan = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      offline = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      localIterators = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      mockInstance = in.readBoolean();
+    }
+    
+    if (in.readBoolean()) {
+      int numColumns = in.readInt();
+      List<String> columns = new ArrayList<String>(numColumns);
+      for (int i = 0; i < numColumns; i++) {
+        columns.add(in.readUTF());
+      }
+      
+      fetchedColumns = InputConfigurator.deserializeFetchedColumns(columns);
+    }
+    
+    if (in.readBoolean()) {
+      String strAuths = in.readUTF();
+      auths = new Authorizations(strAuths.getBytes(Charset.forName("UTF-8")));
+    }
+    
+    if (in.readBoolean()) {
+      principal = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      String tokenClass = in.readUTF();
+      byte[] base64TokenBytes = in.readUTF().getBytes(Charset.forName("UTF-8"));
+      byte[] tokenBytes = Base64.decodeBase64(base64TokenBytes);
+      
+      try {
+        token = CredentialHelper.extractToken(tokenClass, tokenBytes);
+      } catch (AccumuloSecurityException e) {
+        throw new IOException(e);
+      }
+    }
+    
+    if (in.readBoolean()) {
+      instanceName = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      zooKeepers = in.readUTF();
+    }
+    
+    if (in.readBoolean()) {
+      level = Level.toLevel(in.readInt());
+    }
+  }
+
+  public void write(DataOutput out) throws IOException {
+    range.write(out);
+    out.writeInt(locations.length);
+    for (int i = 0; i < locations.length; ++i)
+      out.writeUTF(locations[i]);
+    
+    out.writeBoolean(null != isolatedScan);
+    if (null != isolatedScan) {
+      out.writeBoolean(isolatedScan);
+    }
+    
+    out.writeBoolean(null != offline);
+    if (null != offline) {
+      out.writeBoolean(offline);
+    }
+    
+    out.writeBoolean(null != localIterators);
+    if (null != localIterators) {
+      out.writeBoolean(localIterators);
+    }
+    
+    out.writeBoolean(null != mockInstance);
+    if (null != mockInstance) {
+      out.writeBoolean(mockInstance);
+    }
+    
+    out.writeBoolean(null != fetchedColumns);
+    if (null != fetchedColumns) {
+      String[] cols = InputConfigurator.serializeColumns(fetchedColumns);
+      out.writeInt(cols.length);
+      for (String col : cols) {
+        out.writeUTF(col);
+      }
+    }
+    
+    out.writeBoolean(null != auths);
+    if (null != auths) {
+      out.writeUTF(auths.serialize());
+    }
+    
+    out.writeBoolean(null != principal);
+    if (null != principal) {
+      out.writeUTF(principal);
+    }
+    
+    out.writeBoolean(null != token);
+    if (null != token) {
+      out.writeUTF(token.getClass().getCanonicalName());
+      try {
+        out.writeUTF(CredentialHelper.tokenAsBase64(token));
+      } catch (AccumuloSecurityException e) {
+        throw new IOException(e);
+      }
+    }
+    
+    out.writeBoolean(null != instanceName);
+    if (null != instanceName) {
+      out.writeUTF(instanceName);
+    }
+    
+    out.writeBoolean(null != zooKeepers);
+    if (null != zooKeepers) {
+      out.writeUTF(zooKeepers);
+    }
+    
+    out.writeBoolean(null != level);
+    if (null != level) {
+      out.writeInt(level.toInt());
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder(128);
+    sb.append("Range: ").append(range);
+    sb.append(" Locations: ").append(locations);
+    sb.append(" Table: ").append(table);
+    // TODO finish building of string
+    return sb.toString();
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+  
+  public Instance getInstance() {
+    if (null == instanceName) {
+      return null;
+    }
+    
+    if (isMockInstance()) {  
+      return new MockInstance(getInstanceName());
+    }
+    
+    if (null == zooKeepers) {
+      return null;
+    }
+    
+    return new ZooKeeperInstance(getInstanceName(), getZooKeepers());
+  }
+
+  public String getInstanceName() {
+    return instanceName;
+  }
+
+  public void setInstanceName(String instanceName) {
+    this.instanceName = instanceName;
+  }
+
+  public String getZooKeepers() {
+    return zooKeepers;
+  }
+
+  public void setZooKeepers(String zooKeepers) {
+    this.zooKeepers = zooKeepers;
+  }
+
+  public String getPrincipal() {
+    return principal;
+  }
+
+  public void setPrincipal(String principal) {
+    this.principal = principal;
+  }
+  
+  public AuthenticationToken getToken() {
+    return token;
+  }
+  
+  public void setToken(AuthenticationToken token) {
+    this.token = token;;
+  }
+
+  public Boolean isOffline() {
+    return offline;
+  }
+
+  public void setOffline(Boolean offline) {
+    this.offline = offline;
+  }
+
+  public void setLocations(String[] locations) {
+    this.locations = locations;
+  }
+
+  public Boolean isMockInstance() {
+    return mockInstance;
+  }
+
+  public void setMockInstance(Boolean mockInstance) {
+    this.mockInstance = mockInstance;
+  }
+
+  public Boolean isIsolatedScan() {
+    return isolatedScan;
+  }
+
+  public void setIsolatedScan(Boolean isolatedScan) {
+    this.isolatedScan = isolatedScan;
+  }
+
+  public Authorizations getAuths() {
+    return auths;
+  }
+
+  public void setAuths(Authorizations auths) {
+    this.auths = auths;
+  }
+
+  public void setRange(Range range) {
+    this.range = range;
+  }
+
+  public Boolean usesLocalIterators() {
+    return localIterators;
+  }
+
+  public void setUsesLocalIterators(Boolean localIterators) {
+    this.localIterators = localIterators;
+  }
+
+  public Set<Pair<Text,Text>> getFetchedColumns() {
+    return fetchedColumns;
+  }
+
+  public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
+    this.fetchedColumns = fetchedColumns;
+  }
+
+  public List<IteratorSetting> getIterators() {
+    return iterators;
+  }
+
+  public void setIterators(List<IteratorSetting> iterators) {
+    this.iterators = iterators;
+  }
+
+  public Level getLogLevel() {
+    return level;
+  }
+  
+  public void setLogLevel(Level level) {
+    this.level = level;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/03b065d2/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index 4e5b5a8..5e246c4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -16,17 +16,13 @@
  */
 package org.apache.accumulo.core.client.mapreduce;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.lang.reflect.Method;
-import java.math.BigInteger;
 import java.net.InetAddress;
 import java.net.URLDecoder;
 import java.net.URLEncoder;
 import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -58,7 +54,6 @@ import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.PartialKey;
@@ -73,7 +68,6 @@ import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
@@ -808,7 +802,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    */
   @Override
   public List<InputSplit> getSplits(JobContext context) throws IOException {
-    log.setLevel(getLogLevel(context));
+    Level logLevel = getLogLevel(context);
+    log.setLevel(logLevel);
+    
     validateOptions(context);
     
     String tableName = getInputTableName(context);
@@ -822,16 +818,17 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     Set<Pair<Text,Text>> fetchedColumns = getFetchedColumns(context);
     Authorizations auths = getScanAuthorizations(context);
     String principal = getPrincipal(context);
+    String tokenClass = getTokenClass(context);
+    byte[] tokenBytes = getToken(context);
     
     AuthenticationToken token;
     try {
-       token = CredentialHelper.extractToken(getTokenClass(context), getToken(context));
+       token = CredentialHelper.extractToken(tokenClass, tokenBytes);
     } catch (AccumuloSecurityException e) {
       throw new IOException(e);
     }
     
     List<IteratorSetting> iterators = getIterators(context);
-    Level logLevel = getLogLevel(context);
     
     if (ranges.isEmpty()) {
       ranges = new ArrayList<Range>(1);
@@ -854,9 +851,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         tl = getTabletLocator(context);
         // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
         tl.invalidateCache();
-        while (!tl.binRanges(ranges, binnedRanges,
-            new TCredentials(getPrincipal(context), getTokenClass(context), ByteBuffer.wrap(getToken(context)), getInstance(context).getInstanceID()))
-            .isEmpty()) {
+        while (!tl.binRanges(ranges, binnedRanges, new TCredentials(principal, tokenClass, ByteBuffer.wrap(tokenBytes), instance.getInstanceID())).isEmpty()) {
           if (!(instance instanceof MockInstance)) {
             if (tableId == null)
               tableId = Tables.getTableId(instance, tableName);


[10/19] git commit: ACCUMULO-1854 Get the mapreduce unit tests working again

Posted by el...@apache.org.
ACCUMULO-1854 Get the mapreduce unit tests working again


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/e08736d7
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/e08736d7
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/e08736d7

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: e08736d7900b22d83a0e28c93696f6ef20086942
Parents: f4e4c39
Author: Josh Elser <el...@apache.org>
Authored: Thu Nov 21 21:26:09 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Nov 21 21:26:09 2013 -0500

----------------------------------------------------------------------
 .../core/client/mapreduce/InputFormatBase.java  |  38 +-
 .../core/client/mapreduce/RangeInputSplit.java  |  69 ---
 .../mapreduce/AccumuloInputFormatTest.java      | 159 ++++++
 .../mapreduce/AccumuloInputFormatTest1.java     | 534 -------------------
 .../client/mapreduce/RangeInputSplitTest.java   |  19 +-
 5 files changed, 201 insertions(+), 618 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e08736d7/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index e17b46d..4e5b5a8 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -814,6 +814,24 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     String tableName = getInputTableName(context);
     boolean autoAdjust = getAutoAdjustRanges(context);
     List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(context)) : getRanges(context);
+    Instance instance = getInstance(context);
+    boolean offline = isOfflineScan(context);
+    boolean isolated = isIsolated(context);
+    boolean localIterators = usesLocalIterators(context);
+    boolean mockInstance = (null != instance && MockInstance.class.equals(instance.getClass()));
+    Set<Pair<Text,Text>> fetchedColumns = getFetchedColumns(context);
+    Authorizations auths = getScanAuthorizations(context);
+    String principal = getPrincipal(context);
+    
+    AuthenticationToken token;
+    try {
+       token = CredentialHelper.extractToken(getTokenClass(context), getToken(context));
+    } catch (AccumuloSecurityException e) {
+      throw new IOException(e);
+    }
+    
+    List<IteratorSetting> iterators = getIterators(context);
+    Level logLevel = getLogLevel(context);
     
     if (ranges.isEmpty()) {
       ranges = new ArrayList<Range>(1);
@@ -832,7 +850,6 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           binnedRanges = binOfflineTable(context, tableName, ranges);
         }
       } else {
-        Instance instance = getInstance(context);
         String tableId = null;
         tl = getTabletLocator(context);
         // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
@@ -896,6 +913,25 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     if (!autoAdjust)
       for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
         splits.add(new RangeInputSplit(entry.getKey(), entry.getValue().toArray(new String[0])));
+    
+    for (InputSplit inputSplit : splits) {
+      RangeInputSplit split = (RangeInputSplit) inputSplit;
+
+      split.setTable(tableName);
+      split.setOffline(offline);
+      split.setIsolatedScan(isolated);
+      split.setUsesLocalIterators(localIterators);
+      split.setMockInstance(mockInstance);
+      split.setFetchedColumns(fetchedColumns);
+      split.setPrincipal(principal);
+      split.setToken(token);
+      split.setInstanceName(instance.getInstanceName());
+      split.setZooKeepers(instance.getZooKeepers());
+      split.setAuths(auths);
+      split.setIterators(iterators);
+      split.setLogLevel(logLevel);
+    }
+    
     return splits;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e08736d7/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index 67b839b..6decdc6 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -53,7 +53,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
   private Range range;
   private String[] locations;
   private String table, instanceName, zooKeepers, principal;
-  private String rowRegex, colfamRegex, colqualRegex, valueRegex;
   private AuthenticationToken token;
   private Boolean offline, mockInstance, isolatedScan, localIterators;
   private Authorizations auths;
@@ -164,22 +163,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
     }
     
     if (in.readBoolean()) {
-      rowRegex = in.readUTF();
-    }
-    
-    if (in.readBoolean()) {
-      colfamRegex = in.readUTF();
-    }
-    
-    if (in.readBoolean()) {
-      colqualRegex = in.readUTF();
-    }
-    
-    if (in.readBoolean()) {
-      valueRegex = in.readUTF();
-    }
-    
-    if (in.readBoolean()) {
       int numColumns = in.readInt();
       List<String> columns = new ArrayList<String>(numColumns);
       for (int i = 0; i < numColumns; i++) {
@@ -248,26 +231,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
       out.writeBoolean(mockInstance);
     }
     
-    out.writeBoolean(null != rowRegex);
-    if (null != rowRegex) {
-      out.writeUTF(rowRegex);
-    }
-    
-    out.writeBoolean(null != colfamRegex);
-    if (null != colfamRegex) {
-      out.writeUTF(colfamRegex);
-    }
-    
-    out.writeBoolean(null != colqualRegex);
-    if (null != colqualRegex) {
-      out.writeUTF(colqualRegex);
-    }
-    
-    out.writeBoolean(null != valueRegex);
-    if (null != valueRegex) {
-      out.writeUTF(valueRegex);
-    }
-    
     out.writeBoolean(null != fetchedColumns);
     if (null != fetchedColumns) {
       String[] cols = InputConfigurator.serializeColumns(fetchedColumns);
@@ -391,38 +354,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
     this.locations = locations;
   }
 
-  public String getRowRegex() {
-    return rowRegex;
-  }
-
-  public void setRowRegex(String rowRegex) {
-    this.rowRegex = rowRegex;
-  }
-
-  public String getColfamRegex() {
-    return colfamRegex;
-  }
-
-  public void setColfamRegex(String colfamRegex) {
-    this.colfamRegex = colfamRegex;
-  }
-
-  public String getColqualRegex() {
-    return colqualRegex;
-  }
-
-  public void setColqualRegex(String colqualRegex) {
-    this.colqualRegex = colqualRegex;
-  }
-
-  public String getValueRegex() {
-    return valueRegex;
-  }
-
-  public void setValueRegex(String valueRegex) {
-    this.valueRegex = valueRegex;
-  }
-
   public Boolean isMockInstance() {
     return mockInstance;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e08736d7/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
index 0ee03a2..f9ccdf1 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
@@ -23,11 +23,14 @@ import static org.junit.Assert.assertTrue;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
 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.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -36,16 +39,24 @@ 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.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.util.Pair;
 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.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class AccumuloInputFormatTest {
@@ -318,4 +329,152 @@ public class AccumuloInputFormatTest {
     assertNull(e1);
     assertNull(e2);
   }
+  
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testCorrectRangeInputSplits() throws Exception {
+    Job job = new Job(new Configuration(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
+
+    String username = "user", table = "table", instance = "instance";
+    PasswordToken password = new PasswordToken("password");
+    Authorizations auths = new Authorizations("foo");
+    Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
+    boolean isolated = true, localIters = true;
+    Level level = Level.WARN;
+
+    Instance inst = new MockInstance(instance);
+    Connector connector = inst.getConnector(username, password);
+    connector.tableOperations().create(table);
+
+    AccumuloInputFormat.setConnectorInfo(job, username, password);
+    AccumuloInputFormat.setInputTableName(job, table);
+    AccumuloInputFormat.setScanAuthorizations(job, auths);
+    AccumuloInputFormat.setMockInstance(job, instance);
+    AccumuloInputFormat.setScanIsolation(job, isolated);
+    AccumuloInputFormat.setLocalIterators(job, localIters);
+    AccumuloInputFormat.fetchColumns(job, fetchColumns);
+    AccumuloInputFormat.setLogLevel(job, level);
+    
+    AccumuloInputFormat aif = new AccumuloInputFormat();
+    
+    List<InputSplit> splits = aif.getSplits(job);
+    
+    Assert.assertEquals(1, splits.size());
+    
+    InputSplit split = splits.get(0);
+    
+    Assert.assertEquals(RangeInputSplit.class, split.getClass());
+    
+    RangeInputSplit risplit = (RangeInputSplit) split;
+    
+    Assert.assertEquals(username, risplit.getPrincipal());
+    Assert.assertEquals(table, risplit.getTable());
+    Assert.assertEquals(password, risplit.getToken());
+    Assert.assertEquals(auths, risplit.getAuths());
+    Assert.assertEquals(instance, risplit.getInstanceName());
+    Assert.assertEquals(isolated, risplit.isIsolatedScan());
+    Assert.assertEquals(localIters, risplit.usesLocalIterators());
+    Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());
+    Assert.assertEquals(level, risplit.getLogLevel());
+  }
+  
+  static class TestMapper extends Mapper<Key,Value,Key,Value> {
+    Key key = null;
+    int count = 0;
+
+    @Override
+    protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
+      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));
+      key = new Key(k);
+      count++;
+    }
+  }
+
+  @Test
+  public void testPartialInputSplitDelegationToConfiguration() throws Exception {
+    String user = "testPartialInputSplitUser";
+    PasswordToken password = new PasswordToken("");
+    
+    MockInstance mockInstance = new MockInstance("testPartialInputSplitDelegationToConfiguration");
+    Connector c = mockInstance.getConnector(user, password);
+    c.tableOperations().create("testtable");
+    BatchWriter bw = c.createBatchWriter("testtable", 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();
+
+    Job job = new Job(new Configuration());
+    job.setInputFormatClass(AccumuloInputFormat.class);
+    job.setMapperClass(TestMapper.class);
+    job.setNumReduceTasks(0);
+    AccumuloInputFormat.setConnectorInfo(job, user, password);
+    AccumuloInputFormat.setInputTableName(job, "testtable");
+    AccumuloInputFormat.setScanAuthorizations(job, new Authorizations());
+    AccumuloInputFormat.setMockInstance(job, "testPartialInputSplitDelegationToConfiguration");
+
+    AccumuloInputFormat input = new AccumuloInputFormat();
+    List<InputSplit> splits = input.getSplits(job);
+    assertEquals(splits.size(), 1);
+
+    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
+    
+    RangeInputSplit emptySplit = new RangeInputSplit();
+    
+    // Using an empty split should fall back to the information in the Job's Configuration
+    TaskAttemptID id = new TaskAttemptID();
+    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
+    RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
+    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
+    reader.initialize(emptySplit, context);
+    mapper.run(context);
+  }
+
+  @Test(expected = IOException.class)
+  public void testPartialFailedInputSplitDelegationToConfiguration() throws Exception {
+    String user = "testPartialFailedInputSplit";
+    PasswordToken password = new PasswordToken("");
+    
+    MockInstance mockInstance = new MockInstance("testPartialFailedInputSplitDelegationToConfiguration");
+    Connector c = mockInstance.getConnector(user, password);
+    c.tableOperations().create("testtable");
+    BatchWriter bw = c.createBatchWriter("testtable", 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();
+
+    Job job = new Job(new Configuration());
+    job.setInputFormatClass(AccumuloInputFormat.class);
+    job.setMapperClass(TestMapper.class);
+    job.setNumReduceTasks(0);
+    AccumuloInputFormat.setConnectorInfo(job, user, password);
+    AccumuloInputFormat.setInputTableName(job, "testtable");
+    AccumuloInputFormat.setMockInstance(job, "testPartialFailedInputSplitDelegationToConfiguration");
+
+    AccumuloInputFormat input = new AccumuloInputFormat();
+    List<InputSplit> splits = input.getSplits(job);
+    assertEquals(splits.size(), 1);
+
+    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
+    
+    RangeInputSplit emptySplit = new RangeInputSplit();
+    emptySplit.setPrincipal("root");
+    emptySplit.setToken(new PasswordToken("anythingelse"));
+    
+    // Using an empty split should fall back to the information in the Job's Configuration
+    TaskAttemptID id = new TaskAttemptID();
+    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
+    RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
+    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
+    reader.initialize(emptySplit, context);
+    mapper.run(context);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e08736d7/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest1.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest1.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest1.java
deleted file mode 100644
index 7239b01..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest1.java
+++ /dev/null
@@ -1,534 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.regex.Pattern;
-
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIterator;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.AccumuloIteratorOption;
-import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RegexType;
-import org.apache.accumulo.core.client.mock.MockInstance;
-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.WholeRowIterator;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.log4j.Level;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class AccumuloInputFormatTest {
-
-  @After
-  public void tearDown() throws Exception {}
-
-  /**
-   * Test basic setting & getting of max versions.
-   * 
-   * @throws IOException
-   *           Signals that an I/O exception has occurred.
-   */
-  @Test
-  public void testMaxVersions() throws IOException {
-    JobContext job = new JobContext(new Configuration(), new JobID());
-    AccumuloInputFormat.setMaxVersions(job.getConfiguration(), 1);
-    int version = AccumuloInputFormat.getMaxVersions(job.getConfiguration());
-    assertEquals(1, version);
-  }
-
-  /**
-   * Test max versions with an invalid value.
-   * 
-   * @throws IOException
-   *           Signals that an I/O exception has occurred.
-   */
-  @Test(expected = IOException.class)
-  public void testMaxVersionsLessThan1() throws IOException {
-    JobContext job = new JobContext(new Configuration(), new JobID());
-    AccumuloInputFormat.setMaxVersions(job.getConfiguration(), 0);
-  }
-
-  /**
-   * Test no max version configured.
-   */
-  @Test
-  public void testNoMaxVersion() {
-    JobContext job = new JobContext(new Configuration(), new JobID());
-    assertEquals(-1, AccumuloInputFormat.getMaxVersions(job.getConfiguration()));
-  }
-
-  /**
-   * Check that the iterator configuration is getting stored in the Job conf correctly.
-   */
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testSetIterator() {
-    JobContext job = new JobContext(new Configuration(), new JobID());
-
-    AccumuloInputFormat.setIterator(job, 1, "org.apache.accumulo.core.iterators.WholeRowIterator", "WholeRow");
-    Configuration conf = job.getConfiguration();
-    String iterators = conf.get("AccumuloInputFormat.iterators");
-    assertEquals("1:org.apache.accumulo.core.iterators.WholeRowIterator:WholeRow", iterators);
-  }
-
-  @Test
-  public void testAddIterator() {
-    JobContext job = new JobContext(new Configuration(), new JobID());
-
-    AccumuloInputFormat.addIterator(job.getConfiguration(), new IteratorSetting(1, "WholeRow", WholeRowIterator.class));
-    AccumuloInputFormat.addIterator(job.getConfiguration(), 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.getConfiguration(), iter);
-
-    List<AccumuloIterator> list = AccumuloInputFormat.getIterators(job.getConfiguration());
-
-    // Check the list size
-    assertTrue(list.size() == 3);
-
-    // Walk the list and make sure our settings are correct
-    AccumuloIterator setting = list.get(0);
-    assertEquals(1, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.user.WholeRowIterator", setting.getIteratorClass());
-    assertEquals("WholeRow", setting.getIteratorName());
-
-    setting = list.get(1);
-    assertEquals(2, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
-    assertEquals("Versions", setting.getIteratorName());
-
-    setting = list.get(2);
-    assertEquals(3, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
-    assertEquals("Count", setting.getIteratorName());
-
-    List<AccumuloIteratorOption> iteratorOptions = AccumuloInputFormat.getIteratorOptions(job.getConfiguration());
-    assertEquals(2, iteratorOptions.size());
-    assertEquals("Count", iteratorOptions.get(0).getIteratorName());
-    assertEquals("Count", iteratorOptions.get(1).getIteratorName());
-    assertEquals("v1", iteratorOptions.get(0).getKey());
-    assertEquals("1", iteratorOptions.get(0).getValue());
-    assertEquals("junk", iteratorOptions.get(1).getKey());
-    assertEquals("\0omg:!\\xyzzy", iteratorOptions.get(1).getValue());
-  }
-
-  /**
-   * 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);
-    Job job = new Job();
-    AccumuloInputFormat.addIterator(job.getConfiguration(), someSetting);
-
-    final String rawConfigOpt = new AccumuloIteratorOption("iterator", key, value).toString();
-
-    assertEquals(rawConfigOpt, job.getConfiguration().get("AccumuloInputFormat.iterators.options"));
-
-    List<AccumuloIteratorOption> opts = AccumuloInputFormat.getIteratorOptions(job.getConfiguration());
-    assertEquals(1, opts.size());
-    assertEquals(opts.get(0).getKey(), key);
-    assertEquals(opts.get(0).getValue(), value);
-
-    someSetting.addOption(key + "2", value);
-    someSetting.setPriority(2);
-    someSetting.setName("it2");
-    AccumuloInputFormat.addIterator(job.getConfiguration(), someSetting);
-    opts = AccumuloInputFormat.getIteratorOptions(job.getConfiguration());
-    assertEquals(3, opts.size());
-    for (AccumuloIteratorOption opt : opts) {
-      assertEquals(opt.getKey().substring(0, key.length()), key);
-      assertEquals(opt.getValue(), value);
-    }
-  }
-
-  /**
-   * Test getting iterator settings for multiple iterators set
-   */
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testGetIteratorSettings() {
-    JobContext job = new JobContext(new Configuration(), new JobID());
-
-    AccumuloInputFormat.setIterator(job, 1, "org.apache.accumulo.core.iterators.WholeRowIterator", "WholeRow");
-    AccumuloInputFormat.setIterator(job, 2, "org.apache.accumulo.core.iterators.VersioningIterator", "Versions");
-    AccumuloInputFormat.setIterator(job, 3, "org.apache.accumulo.core.iterators.CountingIterator", "Count");
-
-    List<AccumuloIterator> list = AccumuloInputFormat.getIterators(job);
-
-    // Check the list size
-    assertTrue(list.size() == 3);
-
-    // Walk the list and make sure our settings are correct
-    AccumuloIterator setting = list.get(0);
-    assertEquals(1, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.WholeRowIterator", setting.getIteratorClass());
-    assertEquals("WholeRow", setting.getIteratorName());
-
-    setting = list.get(1);
-    assertEquals(2, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
-    assertEquals("Versions", setting.getIteratorName());
-
-    setting = list.get(2);
-    assertEquals(3, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
-    assertEquals("Count", setting.getIteratorName());
-
-  }
-
-  /**
-   * Check that the iterator options are getting stored in the Job conf correctly.
-   */
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testSetIteratorOption() {
-    JobContext job = new JobContext(new Configuration(), new JobID());
-    AccumuloInputFormat.setIteratorOption(job, "someIterator", "aKey", "aValue");
-
-    Configuration conf = job.getConfiguration();
-    String options = conf.get("AccumuloInputFormat.iterators.options");
-    assertEquals(new String("someIterator:aKey:aValue"), options);
-  }
-
-  /**
-   * Test getting iterator options for multiple options set
-   */
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testGetIteratorOption() {
-    JobContext job = new JobContext(new Configuration(), new JobID());
-
-    AccumuloInputFormat.setIteratorOption(job, "iterator1", "key1", "value1");
-    AccumuloInputFormat.setIteratorOption(job, "iterator2", "key2", "value2");
-    AccumuloInputFormat.setIteratorOption(job, "iterator3", "key3", "value3");
-
-    List<AccumuloIteratorOption> list = AccumuloInputFormat.getIteratorOptions(job);
-
-    // Check the list size
-    assertEquals(3, list.size());
-
-    // Walk the list and make sure all the options are correct
-    AccumuloIteratorOption option = list.get(0);
-    assertEquals("iterator1", option.getIteratorName());
-    assertEquals("key1", option.getKey());
-    assertEquals("value1", option.getValue());
-
-    option = list.get(1);
-    assertEquals("iterator2", option.getIteratorName());
-    assertEquals("key2", option.getKey());
-    assertEquals("value2", option.getValue());
-
-    option = list.get(2);
-    assertEquals("iterator3", option.getIteratorName());
-    assertEquals("key3", option.getKey());
-    assertEquals("value3", option.getValue());
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testSetRegex() {
-    JobContext job = new JobContext(new Configuration(), new JobID());
-
-    String regex = ">\"*%<>\'\\";
-
-    AccumuloInputFormat.setRegex(job, RegexType.ROW, regex);
-
-    assertTrue(regex.equals(AccumuloInputFormat.getRegex(job, RegexType.ROW)));
-  }
-
-  static class TestMapper extends Mapper<Key,Value,Key,Value> {
-    Key key = null;
-    int count = 0;
-
-    @Override
-    protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
-      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));
-      key = new Key(k);
-      count++;
-    }
-  }
-
-  @Test
-  public void testMap() throws Exception {
-    MockInstance mockInstance = new MockInstance("testmapinstance");
-    Connector c = mockInstance.getConnector("root", new byte[] {});
-    c.tableOperations().create("testtable");
-    BatchWriter bw = c.createBatchWriter("testtable", 10000L, 1000L, 4);
-    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();
-
-    Job job = new Job(new Configuration());
-    job.setInputFormatClass(AccumuloInputFormat.class);
-    job.setMapperClass(TestMapper.class);
-    job.setNumReduceTasks(0);
-    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
-    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
-
-    AccumuloInputFormat input = new AccumuloInputFormat();
-    List<InputSplit> splits = input.getSplits(job);
-    assertEquals(splits.size(), 1);
-
-    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
-    for (InputSplit split : splits) {
-      RangeInputSplit risplit = (RangeInputSplit) split;
-      Assert.assertEquals("root", risplit.getUsername());
-      Assert.assertArrayEquals(new byte[0], risplit.getPassword());
-      Assert.assertEquals("testtable", risplit.getTable());
-      Assert.assertEquals(new Authorizations(), risplit.getAuths());
-      Assert.assertEquals("testmapinstance", risplit.getInstanceName());
-      
-      TaskAttemptID id = new TaskAttemptID();
-      TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
-      RecordReader<Key,Value> reader = input.createRecordReader(split, attempt);
-      Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, split);
-      reader.initialize(split, context);
-      mapper.run(context);
-    }
-  }
-
-  @Test
-  public void testSimple() throws Exception {
-    MockInstance mockInstance = new MockInstance("testmapinstance");
-    Connector c = mockInstance.getConnector("root", new byte[] {});
-    c.tableOperations().create("testtable2");
-    BatchWriter bw = c.createBatchWriter("testtable2", 10000L, 1000L, 4);
-    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();
-
-    JobContext job = new JobContext(new Configuration(), new JobID());
-    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable2", new Authorizations());
-    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
-    AccumuloInputFormat input = new AccumuloInputFormat();
-    RangeInputSplit ris = new RangeInputSplit();
-    TaskAttemptContext tac = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID());
-    RecordReader<Key,Value> rr = input.createRecordReader(ris, tac);
-    rr.initialize(ris, tac);
-
-    TestMapper mapper = new TestMapper();
-    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), tac.getTaskAttemptID(), rr, null, null, null, ris);
-    while (rr.nextKeyValue()) {
-      mapper.map(rr.getCurrentKey(), rr.getCurrentValue(), context);
-    }
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testRegex() throws Exception {
-    MockInstance mockInstance = new MockInstance("testmapinstance");
-    Connector c = mockInstance.getConnector("root", new byte[] {});
-    c.tableOperations().create("testtable3");
-    BatchWriter bw = c.createBatchWriter("testtable3", 10000L, 1000L, 4);
-    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();
-
-    JobContext job = new JobContext(new Configuration(), new JobID());
-    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable3", new Authorizations());
-    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
-    final String regex = ".*1.*";
-    AccumuloInputFormat.setRegex(job, RegexType.ROW, regex);
-    AccumuloInputFormat input = new AccumuloInputFormat();
-    RangeInputSplit ris = new RangeInputSplit();
-    TaskAttemptContext tac = new TaskAttemptContext(job.getConfiguration(), new TaskAttemptID());
-    RecordReader<Key,Value> rr = input.createRecordReader(ris, tac);
-    rr.initialize(ris, tac);
-
-    Pattern p = Pattern.compile(regex);
-    while (rr.nextKeyValue()) {
-      Assert.assertTrue(p.matcher(rr.getCurrentKey().getRow().toString()).matches());
-    }
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testCorrectRangeInputSplits() throws Exception {
-    JobContext job = new JobContext(new Configuration(), new JobID());
-
-    String username = "user", table = "table", rowRegex = "row.*", colfRegex = "colf.*", colqRegex = "colq.*";
-    String valRegex = "val.*", instance = "instance";
-    byte[] password = "password".getBytes();
-    Authorizations auths = new Authorizations("foo");
-    Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
-    boolean isolated = true, localIters = true;
-    int maxVersions = 5;
-    Level level = Level.WARN;
-
-    Instance inst = new MockInstance(instance);
-    Connector connector = inst.getConnector(username, password);
-    connector.tableOperations().create(table);
-
-    AccumuloInputFormat.setInputInfo(job, username, password, table, auths);
-    AccumuloInputFormat.setMockInstance(job, instance);
-    AccumuloInputFormat.setRegex(job, RegexType.ROW, rowRegex);
-    AccumuloInputFormat.setRegex(job, RegexType.COLUMN_FAMILY, colfRegex);
-    AccumuloInputFormat.setRegex(job, RegexType.COLUMN_QUALIFIER, colqRegex);
-    AccumuloInputFormat.setRegex(job, RegexType.VALUE, valRegex);
-    AccumuloInputFormat.setIsolated(job, isolated);
-    AccumuloInputFormat.setLocalIterators(job, localIters);
-    AccumuloInputFormat.setMaxVersions(job, maxVersions);
-    AccumuloInputFormat.fetchColumns(job, fetchColumns);
-    AccumuloInputFormat.setLogLevel(job, level);
-    
-    AccumuloInputFormat aif = new AccumuloInputFormat();
-    
-    List<InputSplit> splits = aif.getSplits(job);
-    
-    Assert.assertEquals(1, splits.size());
-    
-    InputSplit split = splits.get(0);
-    
-    Assert.assertEquals(RangeInputSplit.class, split.getClass());
-    
-    RangeInputSplit risplit = (RangeInputSplit) split;
-    
-    Assert.assertEquals(username, risplit.getUsername());
-    Assert.assertEquals(table, risplit.getTable());
-    Assert.assertArrayEquals(password, risplit.getPassword());
-    Assert.assertEquals(auths, risplit.getAuths());
-    Assert.assertEquals(instance, risplit.getInstanceName());
-    Assert.assertEquals(rowRegex, risplit.getRowRegex());
-    Assert.assertEquals(colfRegex, risplit.getColfamRegex());
-    Assert.assertEquals(colqRegex, risplit.getColqualRegex());
-    Assert.assertEquals(valRegex, risplit.getValueRegex());
-    Assert.assertEquals(isolated, risplit.isIsolatedScan());
-    Assert.assertEquals(localIters, risplit.usesLocalIterators());
-    Assert.assertEquals(maxVersions, risplit.getMaxVersions().intValue());
-    Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());
-    Assert.assertEquals(level, risplit.getLogLevel());
-  }
-
-  @Test
-  public void testPartialInputSplitDelegationToConfiguration() throws Exception {
-    MockInstance mockInstance = new MockInstance("testPartialInputSplitDelegationToConfiguration");
-    Connector c = mockInstance.getConnector("root", new byte[] {});
-    c.tableOperations().create("testtable");
-    BatchWriter bw = c.createBatchWriter("testtable", 10000L, 1000L, 4);
-    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();
-
-    Job job = new Job(new Configuration());
-    job.setInputFormatClass(AccumuloInputFormat.class);
-    job.setMapperClass(TestMapper.class);
-    job.setNumReduceTasks(0);
-    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
-    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testPartialInputSplitDelegationToConfiguration");
-
-    AccumuloInputFormat input = new AccumuloInputFormat();
-    List<InputSplit> splits = input.getSplits(job);
-    assertEquals(splits.size(), 1);
-
-    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
-    
-    RangeInputSplit emptySplit = new RangeInputSplit();
-    
-    // Using an empty split should fall back to the information in the Job's Configuration
-    TaskAttemptID id = new TaskAttemptID();
-    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
-    RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
-    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
-    reader.initialize(emptySplit, context);
-    mapper.run(context);
-  }
-
-  @Test(expected = IOException.class)
-  public void testPartialFailedInputSplitDelegationToConfiguration() throws Exception {
-    MockInstance mockInstance = new MockInstance("testPartialFailedInputSplitDelegationToConfiguration");
-    Connector c = mockInstance.getConnector("root", new byte[] {});
-    c.tableOperations().create("testtable");
-    BatchWriter bw = c.createBatchWriter("testtable", 10000L, 1000L, 4);
-    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();
-
-    Job job = new Job(new Configuration());
-    job.setInputFormatClass(AccumuloInputFormat.class);
-    job.setMapperClass(TestMapper.class);
-    job.setNumReduceTasks(0);
-    AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
-    AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testPartialFailedInputSplitDelegationToConfiguration");
-
-    AccumuloInputFormat input = new AccumuloInputFormat();
-    List<InputSplit> splits = input.getSplits(job);
-    assertEquals(splits.size(), 1);
-
-    TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
-    
-    RangeInputSplit emptySplit = new RangeInputSplit();
-    emptySplit.setUsername("root");
-    emptySplit.setPassword("anythingelse".getBytes());
-    
-    // Using an empty split should fall back to the information in the Job's Configuration
-    TaskAttemptID id = new TaskAttemptID();
-    TaskAttemptContext attempt = new TaskAttemptContext(job.getConfiguration(), id);
-    RecordReader<Key,Value> reader = input.createRecordReader(emptySplit, attempt);
-    Mapper<Key,Value,Key,Value>.Context context = mapper.new Context(job.getConfiguration(), id, reader, null, null, null, emptySplit);
-    reader.initialize(emptySplit, context);
-    mapper.run(context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e08736d7/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
index 22fb6e1..f6c604f 100644
--- 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
@@ -9,6 +9,7 @@ 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;
@@ -53,14 +54,9 @@ public class RangeInputSplitTest {
     split.setOffline(true);
     split.setIsolatedScan(true);
     split.setUsesLocalIterators(true);
-    split.setMaxVersions(5);
-    split.setRowRegex("row");
-    split.setColfamRegex("colf");
-    split.setColqualRegex("colq");
-    split.setValueRegex("value");
     split.setFetchedColumns(fetchedColumns);
-    split.setPassword("password".getBytes());
-    split.setUsername("root");
+    split.setToken(new PasswordToken("password"));
+    split.setPrincipal("root");
     split.setInstanceName("instance");
     split.setMockInstance(true);
     split.setZooKeepers("localhost");
@@ -83,14 +79,9 @@ public class RangeInputSplitTest {
     Assert.assertEquals(split.isOffline(), newSplit.isOffline());
     Assert.assertEquals(split.isIsolatedScan(), newSplit.isOffline());
     Assert.assertEquals(split.usesLocalIterators(), newSplit.usesLocalIterators());
-    Assert.assertEquals(split.getMaxVersions(), newSplit.getMaxVersions());
-    Assert.assertEquals(split.getRowRegex(), newSplit.getRowRegex());
-    Assert.assertEquals(split.getColfamRegex(), newSplit.getColfamRegex());
-    Assert.assertEquals(split.getColqualRegex(), newSplit.getColqualRegex());
-    Assert.assertEquals(split.getValueRegex(), newSplit.getValueRegex());
     Assert.assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
-    Assert.assertEquals(new String(split.getPassword()), new String(newSplit.getPassword()));
-    Assert.assertEquals(split.getUsername(), newSplit.getUsername());
+    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());


[09/19] git commit: ACCUMULO-1854 Removing unnecessary configurations.

Posted by el...@apache.org.
ACCUMULO-1854 Removing unnecessary configurations.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/f4e4c39d
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/f4e4c39d
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/f4e4c39d

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: f4e4c39de9c599067f4a0b35007d11302c3bd709
Parents: 4dfe3e0
Author: Josh Elser <el...@apache.org>
Authored: Thu Nov 21 20:42:47 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Nov 21 20:42:47 2013 -0500

----------------------------------------------------------------------
 .../core/client/mapreduce/RangeInputSplit.java    | 18 ------------------
 1 file changed, 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4e4c39d/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index d613836..67b839b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -56,7 +56,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
   private String rowRegex, colfamRegex, colqualRegex, valueRegex;
   private AuthenticationToken token;
   private Boolean offline, mockInstance, isolatedScan, localIterators;
-  private Integer maxVersions;
   private Authorizations auths;
   private Set<Pair<Text,Text>> fetchedColumns;
   private List<IteratorSetting> iterators;
@@ -165,10 +164,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
     }
     
     if (in.readBoolean()) {
-      maxVersions = in.readInt();
-    }
-    
-    if (in.readBoolean()) {
       rowRegex = in.readUTF();
     }
     
@@ -253,11 +248,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
       out.writeBoolean(mockInstance);
     }
     
-    out.writeBoolean(null != maxVersions);
-    if (null != maxVersions) {
-      out.writeInt(getMaxVersions());
-    }
-    
     out.writeBoolean(null != rowRegex);
     if (null != rowRegex) {
       out.writeUTF(rowRegex);
@@ -449,14 +439,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
     this.isolatedScan = isolatedScan;
   }
 
-  public Integer getMaxVersions() {
-    return maxVersions;
-  }
-
-  public void setMaxVersions(Integer maxVersions) {
-    this.maxVersions = maxVersions;
-  }
-
   public Authorizations getAuths() {
     return auths;
   }


[06/19] git commit: WIP Merge branch 'ACCUMULO-1854-merge' into ACCUMULO-1854-1.5-merge

Posted by el...@apache.org.
WIP Merge branch 'ACCUMULO-1854-merge' into ACCUMULO-1854-1.5-merge

Conflicts:
	core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
	src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
	src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
	src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
	src/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/16a2e0f0
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/16a2e0f0
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/16a2e0f0

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 16a2e0f0a1b367100355a232b22b735a8c06db1e
Parents: df053b4 45ae55f
Author: Josh Elser <el...@apache.org>
Authored: Thu Nov 21 19:10:43 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Nov 21 19:10:43 2013 -0500

----------------------------------------------------------------------
 .../core/client/mapred/InputFormatBase.java     |  26 +-
 .../client/mapreduce/AccumuloInputFormat.java   |  11 +
 .../core/client/mapreduce/InputFormatBase.java  | 214 +++-----
 .../core/client/mapreduce/RangeInputSplit.java  | 503 +++++++++++++++++
 .../mapreduce/lib/util/InputConfigurator.java   |  31 +-
 .../core/security/CredentialHelper.java         |   2 +-
 .../mapreduce/AccumuloInputFormatTest1.java     | 534 +++++++++++++++++++
 .../client/mapreduce/RangeInputSplitTest.java   | 100 ++++
 8 files changed, 1251 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/16a2e0f0/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
index b502b1f,0000000..8d3d710
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
@@@ -1,843 -1,0 +1,825 @@@
 +/*
 + * 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.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.HashMap;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +
 +import org.apache.accumulo.core.Constants;
 +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.TableDeletedException;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.TableOfflineException;
 +import org.apache.accumulo.core.client.ZooKeeperInstance;
 +import org.apache.accumulo.core.client.impl.OfflineScanner;
 +import org.apache.accumulo.core.client.impl.Tables;
 +import org.apache.accumulo.core.client.impl.TabletLocator;
++import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
 +import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 +import org.apache.accumulo.core.client.mock.MockInstance;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.master.state.tables.TableState;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +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.hadoop.mapred.Reporter;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of K,V pairs.
 + * <p>
 + * Subclasses must implement a {@link #getRecordReader(InputSplit, JobConf, Reporter)} to provide a {@link RecordReader} for K,V.
 + * <p>
 + * A static base class, RecordReaderBase, is provided to retrieve Accumulo {@link Key}/{@link Value} pairs, but one must implement its
 + * {@link RecordReaderBase#next(Object, Object)} to transform them to the desired generic types K,V.
 + * <p>
 + * See {@link AccumuloInputFormat} for an example implementation.
 + */
 +public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
 +  
 +  private static final Class<?> CLASS = AccumuloInputFormat.class;
 +  protected static final Logger log = Logger.getLogger(CLASS);
 +  
 +  /**
 +   * Sets the connector information needed to communicate with Accumulo in this job.
 +   * 
 +   * <p>
 +   * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to provide a charset safe
 +   * conversion to a string, and is not intended to be secure.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param principal
 +   *          a valid Accumulo user name (user must have Table.CREATE permission)
 +   * @param token
 +   *          the user's password
 +   * @throws AccumuloSecurityException
 +   * @since 1.5.0
 +   */
 +  public static void setConnectorInfo(JobConf job, String principal, AuthenticationToken token) throws AccumuloSecurityException {
 +    InputConfigurator.setConnectorInfo(CLASS, job, principal, token);
 +  }
 +  
 +  /**
 +   * 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 serialized token class 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 getTokenClass(JobConf job) {
 +    return InputConfigurator.getTokenClass(CLASS, job);
 +  }
 +  
 +  /**
 +   * Gets the password from the configuration. WARNING: The password 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 context for the configured job
 +   * @return the decoded user password
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
 +   */
 +  protected static byte[] getToken(JobConf job) {
 +    return InputConfigurator.getToken(CLASS, job);
 +  }
 +  
 +  /**
 +   * Configures a {@link ZooKeeperInstance} for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param instanceName
 +   *          the Accumulo instance name
 +   * @param zooKeepers
 +   *          a comma-separated list of zookeeper servers
 +   * @since 1.5.0
 +   */
 +  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
 +    InputConfigurator.setZooKeeperInstance(CLASS, job, instanceName, zooKeepers);
 +  }
 +  
 +  /**
 +   * Configures a {@link MockInstance} for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param instanceName
 +   *          the Accumulo instance name
 +   * @since 1.5.0
 +   */
 +  public static void setMockInstance(JobConf job, String instanceName) {
 +    InputConfigurator.setMockInstance(CLASS, job, instanceName);
 +  }
 +  
 +  /**
 +   * Initializes an Accumulo {@link 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, String, String)
 +   * @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 name of the input table, over which this job will scan.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param tableName
 +   *          the table to use when the tablename is null in the write call
 +   * @since 1.5.0
 +   */
 +  public static void setInputTableName(JobConf job, String tableName) {
 +    InputConfigurator.setInputTableName(CLASS, job, tableName);
 +  }
 +  
 +  /**
 +   * Gets the table name from the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the table name
 +   * @since 1.5.0
 +   * @see #setInputTableName(JobConf, String)
 +   */
 +  protected static String getInputTableName(JobConf job) {
 +    return InputConfigurator.getInputTableName(CLASS, job);
 +  }
 +  
 +  /**
 +   * Sets the {@link 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);
 +  }
 +  
 +  /**
 +   * Sets the input ranges to scan for this job. If not set, the entire table will be scanned.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param ranges
 +   *          the ranges that will be mapped over
 +   * @since 1.5.0
 +   */
 +  public static void setRanges(JobConf job, Collection<Range> ranges) {
 +    InputConfigurator.setRanges(CLASS, job, ranges);
 +  }
 +  
 +  /**
 +   * Gets the ranges to scan over from a job.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return the ranges
 +   * @throws IOException
 +   *           if the ranges have been encoded improperly
 +   * @since 1.5.0
 +   * @see #setRanges(JobConf, Collection)
 +   */
 +  protected static List<Range> getRanges(JobConf job) throws IOException {
 +    return InputConfigurator.getRanges(CLASS, job);
 +  }
 +  
 +  /**
 +   * Restricts the columns that will be mapped over for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param columnFamilyColumnQualifierPairs
 +   *          a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
 +   *          selected. An empty set is the default and is equivalent to scanning the all columns.
 +   * @since 1.5.0
 +   */
 +  public static void fetchColumns(JobConf job, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 +    InputConfigurator.fetchColumns(CLASS, job, columnFamilyColumnQualifierPairs);
 +  }
 +  
 +  /**
 +   * Gets the columns to be mapped over from this job.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return a set of columns
 +   * @since 1.5.0
 +   * @see #fetchColumns(JobConf, Collection)
 +   */
 +  protected static Set<Pair<Text,Text>> getFetchedColumns(JobConf job) {
 +    return InputConfigurator.getFetchedColumns(CLASS, job);
 +  }
 +  
 +  /**
 +   * Encode an iterator on the input for this job.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param cfg
 +   *          the configuration of the iterator
 +   * @since 1.5.0
 +   */
 +  public static void addIterator(JobConf job, IteratorSetting cfg) {
 +    InputConfigurator.addIterator(CLASS, job, cfg);
 +  }
 +  
 +  /**
 +   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return a list of iterators
 +   * @since 1.5.0
 +   * @see #addIterator(JobConf, IteratorSetting)
 +   */
 +  protected static List<IteratorSetting> getIterators(JobConf job) {
 +    return InputConfigurator.getIterators(CLASS, job);
 +  }
 +  
 +  /**
 +   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
 +   * Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
 +   * 
 +   * <p>
 +   * By default, this feature is <b>enabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @see #setRanges(JobConf, Collection)
 +   * @since 1.5.0
 +   */
 +  public static void setAutoAdjustRanges(JobConf job, boolean enableFeature) {
 +    InputConfigurator.setAutoAdjustRanges(CLASS, job, enableFeature);
 +  }
 +  
 +  /**
 +   * Determines whether a configuration has auto-adjust ranges enabled.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return false if the feature is disabled, true otherwise
 +   * @since 1.5.0
 +   * @see #setAutoAdjustRanges(JobConf, boolean)
 +   */
 +  protected static boolean getAutoAdjustRanges(JobConf job) {
 +    return InputConfigurator.getAutoAdjustRanges(CLASS, job);
 +  }
 +  
 +  /**
 +   * Controls the use of the {@link IsolatedScanner} in this job.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.5.0
 +   */
 +  public static void setScanIsolation(JobConf job, boolean enableFeature) {
 +    InputConfigurator.setScanIsolation(CLASS, job, enableFeature);
 +  }
 +  
 +  /**
 +   * Determines whether a configuration has isolation enabled.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.5.0
 +   * @see #setScanIsolation(JobConf, boolean)
 +   */
 +  protected static boolean isIsolated(JobConf job) {
 +    return InputConfigurator.isIsolated(CLASS, job);
 +  }
 +  
 +  /**
 +   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack to be constructed within the Map
 +   * task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be available on the classpath for the task.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.5.0
 +   */
 +  public static void setLocalIterators(JobConf job, boolean enableFeature) {
 +    InputConfigurator.setLocalIterators(CLASS, job, enableFeature);
 +  }
 +  
 +  /**
 +   * Determines whether a configuration uses local iterators.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.5.0
 +   * @see #setLocalIterators(JobConf, boolean)
 +   */
 +  protected static boolean usesLocalIterators(JobConf job) {
 +    return InputConfigurator.usesLocalIterators(CLASS, job);
 +  }
 +  
 +  /**
 +   * <p>
 +   * Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
 +   * table's files. If the table is not offline, then the job will fail. If the table comes online during the map reduce job, it is likely that the job will
 +   * fail.
 +   * 
 +   * <p>
 +   * To use this option, the map reduce user will need access to read the Accumulo directory in HDFS.
 +   * 
 +   * <p>
 +   * Reading the offline table will create the scan time iterator stack in the map process. So any iterators that are configured for the table will need to be
 +   * on the mapper's classpath. The accumulo-site.xml may need to be on the mapper's classpath if HDFS or the Accumulo directory in HDFS are non-standard.
 +   * 
 +   * <p>
 +   * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
 +   * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
 +   * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
 +   * 
 +   * <p>
 +   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
 +   * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.5.0
 +   */
 +  public static void setOfflineTableScan(JobConf job, boolean enableFeature) {
 +    InputConfigurator.setOfflineTableScan(CLASS, job, enableFeature);
 +  }
 +  
 +  /**
 +   * Determines whether a configuration has the offline table scan feature enabled.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.5.0
 +   * @see #setOfflineTableScan(JobConf, boolean)
 +   */
 +  protected static boolean isOfflineScan(JobConf job) {
 +    return InputConfigurator.isOfflineScan(CLASS, job);
 +  }
 +  
 +  /**
 +   * Initializes an Accumulo {@link TabletLocator} based on the configuration.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @return an Accumulo tablet locator
 +   * @throws TableNotFoundException
 +   *           if the table name set on the configuration doesn't exist
 +   * @since 1.5.0
 +   */
 +  protected static TabletLocator getTabletLocator(JobConf job) throws TableNotFoundException {
 +    return InputConfigurator.getTabletLocator(CLASS, job);
 +  }
 +  
 +  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
 +  /**
 +   * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
 +   * 
 +   * @param job
 +   *          the Hadoop context for the configured job
 +   * @throws IOException
 +   *           if the context is improperly configured
 +   * @since 1.5.0
 +   */
 +  protected static void validateOptions(JobConf job) throws IOException {
 +    InputConfigurator.validateOptions(CLASS, job);
 +  }
 +  
 +  /**
 +   * An abstract base class to be used to create {@link RecordReader} instances that convert from Accumulo {@link Key}/{@link 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 RecordReaderBase<K,V> implements RecordReader<K,V> {
 +    protected long numKeysRead;
 +    protected Iterator<Entry<Key,Value>> scannerIterator;
 +    protected RangeInputSplit split;
 +    
 +    /**
 +     * Apply the configured iterators from the configuration to the scanner.
 +     * 
 +     * @param job
 +     *          the Hadoop context for the configured job
 +     * @param scanner
 +     *          the scanner to configure
 +     */
 +    protected void setupIterators(JobConf job, Scanner scanner) {
 +      List<IteratorSetting> iterators = getIterators(job);
 +      for (IteratorSetting iterator : iterators) {
 +        scanner.addScanIterator(iterator);
 +      }
 +    }
 +    
 +    /**
 +     * Initialize a scanner over the given input split using this task attempt configuration.
 +     */
 +    public void initialize(InputSplit inSplit, JobConf job) throws IOException {
 +      Scanner scanner;
 +      split = (RangeInputSplit) inSplit;
 +      log.debug("Initializing input split: " + split.getRange());
 +      Instance instance = getInstance(job);
 +      String user = getPrincipal(job);
 +      String tokenClass = getTokenClass(job);
 +      byte[] password = getToken(job);
 +      Authorizations authorizations = getScanAuthorizations(job);
 +      
 +      try {
 +        log.debug("Creating connector with user: " + user);
 +        Connector conn = instance.getConnector(user, CredentialHelper.extractToken(tokenClass, password));
 +        log.debug("Creating scanner for table: " + getInputTableName(job));
 +        log.debug("Authorizations are: " + authorizations);
 +        if (isOfflineScan(job)) {
 +          scanner = new OfflineScanner(instance, new TCredentials(user, tokenClass, ByteBuffer.wrap(password), instance.getInstanceID()), Tables.getTableId(
 +              instance, getInputTableName(job)), authorizations);
 +        } else {
 +          scanner = conn.createScanner(getInputTableName(job), authorizations);
 +        }
 +        if (isIsolated(job)) {
 +          log.info("Creating isolated scanner");
 +          scanner = new IsolatedScanner(scanner);
 +        }
 +        if (usesLocalIterators(job)) {
 +          log.info("Using local iterators");
 +          scanner = new ClientSideIteratorScanner(scanner);
 +        }
 +        setupIterators(job, scanner);
 +      } catch (Exception e) {
 +        throw new IOException(e);
 +      }
 +      
 +      // setup a scanner within the bounds of this split
 +      for (Pair<Text,Text> c : getFetchedColumns(job)) {
 +        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 tableName, List<Range> ranges) throws TableNotFoundException, AccumuloException,
 +      AccumuloSecurityException {
 +    
 +    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +    
 +    Instance instance = getInstance(job);
 +    Connector conn = instance.getConnector(getPrincipal(job), CredentialHelper.extractToken(getTokenClass(job), getToken(job)));
 +    String tableId = Tables.getTableId(instance, tableName);
 +    
 +    if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
 +      Tables.clearCache(instance);
 +      if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
 +        throw new AccumuloException("Table is online " + tableName + "(" + tableId + ") cannot scan table in offline mode ");
 +      }
 +    }
 +    
 +    for (Range range : ranges) {
 +      Text startRow;
 +      
 +      if (range.getStartKey() != null)
 +        startRow = range.getStartKey().getRow();
 +      else
 +        startRow = new Text();
 +      
 +      Range metadataRange = new Range(new KeyExtent(new Text(tableId), startRow, null).getMetadataEntry(), true, null, false);
 +      Scanner scanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
 +      Constants.METADATA_PREV_ROW_COLUMN.fetch(scanner);
 +      scanner.fetchColumnFamily(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY);
 +      scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
 +      scanner.fetchColumnFamily(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY);
 +      scanner.setRange(metadataRange);
 +      
 +      RowIterator rowIter = new RowIterator(scanner);
 +      
 +      KeyExtent lastExtent = null;
 +      
 +      while (rowIter.hasNext()) {
 +        Iterator<Entry<Key,Value>> row = rowIter.next();
 +        String last = "";
 +        KeyExtent extent = null;
 +        String location = null;
 +        
 +        while (row.hasNext()) {
 +          Entry<Key,Value> entry = row.next();
 +          Key key = entry.getKey();
 +          
 +          if (key.getColumnFamily().equals(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY)) {
 +            last = entry.getValue().toString();
 +          }
 +          
 +          if (key.getColumnFamily().equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY)
 +              || key.getColumnFamily().equals(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY)) {
 +            location = entry.getValue().toString();
 +          }
 +          
 +          if (Constants.METADATA_PREV_ROW_COLUMN.hasColumns(key)) {
 +            extent = new KeyExtent(key.getRow(), entry.getValue());
 +          }
 +          
 +        }
 +        
 +        if (location != null)
 +          return null;
 +        
 +        if (!extent.getTableId().toString().equals(tableId)) {
 +          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
 +        }
 +        
 +        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
 +          throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
 +        }
 +        
 +        Map<KeyExtent,List<Range>> tabletRanges = binnedRanges.get(last);
 +        if (tabletRanges == null) {
 +          tabletRanges = new HashMap<KeyExtent,List<Range>>();
 +          binnedRanges.put(last, tabletRanges);
 +        }
 +        
 +        List<Range> rangeList = tabletRanges.get(extent);
 +        if (rangeList == null) {
 +          rangeList = new ArrayList<Range>();
 +          tabletRanges.put(extent, rangeList);
 +        }
 +        
 +        rangeList.add(range);
 +        
 +        if (extent.getEndRow() == null || range.afterEndKey(new Key(extent.getEndRow()).followingKey(PartialKey.ROW))) {
 +          break;
 +        }
 +        
 +        lastExtent = extent;
 +      }
 +      
 +    }
 +    
 +    return binnedRanges;
 +  }
 +  
 +  /**
 +   * Read the metadata table to get tablets and match up ranges to them.
 +   */
 +  @Override
 +  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
 +    log.setLevel(getLogLevel(job));
 +    validateOptions(job);
 +    
 +    String tableName = getInputTableName(job);
 +    boolean autoAdjust = getAutoAdjustRanges(job);
 +    List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(job)) : getRanges(job);
 +    
 +    if (ranges.isEmpty()) {
 +      ranges = new ArrayList<Range>(1);
 +      ranges.add(new Range());
 +    }
 +    
 +    // get the metadata information for these ranges
 +    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +    TabletLocator tl;
 +    try {
 +      if (isOfflineScan(job)) {
 +        binnedRanges = binOfflineTable(job, tableName, ranges);
 +        while (binnedRanges == null) {
 +          // Some tablets were still online, try again
 +          UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
 +          binnedRanges = binOfflineTable(job, tableName, ranges);
 +        }
 +      } else {
 +        Instance instance = getInstance(job);
 +        String tableId = null;
 +        tl = getTabletLocator(job);
 +        // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
 +        tl.invalidateCache();
 +        while (!tl.binRanges(ranges, binnedRanges,
 +            new TCredentials(getPrincipal(job), getTokenClass(job), ByteBuffer.wrap(getToken(job)), getInstance(job).getInstanceID())).isEmpty()) {
 +          if (!(instance instanceof MockInstance)) {
 +            if (tableId == null)
 +              tableId = Tables.getTableId(instance, tableName);
 +            if (!Tables.exists(instance, tableId))
 +              throw new TableDeletedException(tableId);
 +            if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
 +              throw new TableOfflineException(instance, tableId);
 +          }
 +          binnedRanges.clear();
 +          log.warn("Unable to locate bins for specified ranges. Retrying.");
 +          UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
 +          tl.invalidateCache();
 +        }
 +      }
 +    } catch (Exception e) {
 +      throw new IOException(e);
 +    }
 +    
-     ArrayList<InputSplit> splits = new ArrayList<InputSplit>(ranges.size());
++    ArrayList<RangeInputSplit> splits = new ArrayList<RangeInputSplit>(ranges.size());
 +    HashMap<Range,ArrayList<String>> splitsToAdd = null;
 +    
 +    if (!autoAdjust)
 +      splitsToAdd = new HashMap<Range,ArrayList<String>>();
 +    
 +    HashMap<String,String> hostNameCache = new HashMap<String,String>();
 +    
 +    for (Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
 +      String ip = tserverBin.getKey().split(":", 2)[0];
 +      String location = hostNameCache.get(ip);
 +      if (location == null) {
 +        InetAddress inetAddress = InetAddress.getByName(ip);
 +        location = inetAddress.getHostName();
 +        hostNameCache.put(ip, location);
 +      }
 +      
 +      for (Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
 +        Range ke = extentRanges.getKey().toDataRange();
 +        for (Range r : extentRanges.getValue()) {
 +          if (autoAdjust) {
 +            // divide ranges into smaller ranges, based on the tablets
-             splits.add(new RangeInputSplit(tableName, ke.clip(r), new String[] {location}));
++            splits.add(new RangeInputSplit(ke.clip(r), new String[] {location}));
 +          } else {
 +            // don't divide ranges
 +            ArrayList<String> locations = splitsToAdd.get(r);
 +            if (locations == null)
 +              locations = new ArrayList<String>(1);
 +            locations.add(location);
 +            splitsToAdd.put(r, locations);
 +          }
 +        }
 +      }
 +    }
 +    
 +    if (!autoAdjust)
 +      for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
-         splits.add(new RangeInputSplit(tableName, entry.getKey(), entry.getValue().toArray(new String[0])));
++        splits.add(new RangeInputSplit(entry.getKey(), entry.getValue().toArray(new String[0])));
 +    return splits.toArray(new InputSplit[splits.size()]);
 +  }
 +  
-   /**
-    * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
-    */
-   public static class RangeInputSplit extends org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit implements InputSplit {
-     
-     public RangeInputSplit() {
-       super();
-     }
-     
-     public RangeInputSplit(RangeInputSplit split) throws IOException {
-       super(split);
-     }
-     
-     protected RangeInputSplit(String table, Range range, String[] locations) {
-       super(table, range, locations);
-     }
-     
-   }
-   
 +}

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


[14/19] git commit: ACCUMULO-1854 Merge the mapred and mapreduce RangeInputSplits into one class

Posted by el...@apache.org.
ACCUMULO-1854 Merge the mapred and mapreduce RangeInputSplits into one class


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/67189202
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/67189202
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/67189202

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: 67189202183342888dc9b00b5ef3366e748997d6
Parents: 4f09824
Author: Josh Elser <el...@apache.org>
Authored: Fri Nov 22 13:58:01 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Nov 22 13:58:01 2013 -0500

----------------------------------------------------------------------
 .../core/client/mapred/RangeInputSplit.java     | 387 +------------------
 .../core/client/mapreduce/RangeInputSplit.java  |  15 +-
 2 files changed, 16 insertions(+), 386 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/67189202/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
index 01bf6dc..b35cef5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
@@ -16,401 +16,20 @@
  */
 package org.apache.accumulo.core.client.mapred;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigInteger;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.CredentialHelper;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.InputSplit;
-import org.apache.log4j.Level;
 
 /**
  * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
  */
-public class RangeInputSplit implements InputSplit, Writable {
-  private Range range;
-  private String[] locations;
-  private String table, instanceName, zooKeepers, principal;
-  private AuthenticationToken token;
-  private Boolean offline, mockInstance, isolatedScan, localIterators;
-  private Authorizations auths;
-  private Set<Pair<Text,Text>> fetchedColumns;
-  private List<IteratorSetting> iterators;
-  private Level level;
+public class RangeInputSplit extends org.apache.accumulo.core.client.mapreduce.RangeInputSplit implements InputSplit {
 
   public RangeInputSplit() {
-    range = new Range();
-    locations = new String[0];
+    super();
   }
 
   public RangeInputSplit(Range range, String[] locations) {
-    this.range = range;
-    this.locations = locations;
-  }
-
-  public Range getRange() {
-    return range;
-  }
-
-  private static byte[] extractBytes(ByteSequence seq, int numBytes) {
-    byte[] bytes = new byte[numBytes + 1];
-    bytes[0] = 0;
-    for (int i = 0; i < numBytes; i++) {
-      if (i >= seq.length())
-        bytes[i + 1] = 0;
-      else
-        bytes[i + 1] = seq.byteAt(i);
-    }
-    return bytes;
-  }
-
-  public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
-    int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
-    BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
-    BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
-    BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
-    return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
-  }
-
-  public float getProgress(Key currentKey) {
-    if (currentKey == null)
-      return 0f;
-    if (range.getStartKey() != null && range.getEndKey() != null) {
-      if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
-        // just look at the row progress
-        return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
-      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
-        // just look at the column family progress
-        return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
-      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL) != 0) {
-        // just look at the column qualifier progress
-        return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
-      }
-    }
-    // if we can't figure it out, then claim no progress
-    return 0f;
-  }
-
-  /**
-   * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
-   */
-  public long getLength() throws IOException {
-    Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
-    Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
-    int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
-    long diff = 0;
-
-    byte[] start = startRow.getBytes();
-    byte[] stop = stopRow.getBytes();
-    for (int i = 0; i < maxCommon; ++i) {
-      diff |= 0xff & (start[i] ^ stop[i]);
-      diff <<= Byte.SIZE;
-    }
-
-    if (startRow.getLength() != stopRow.getLength())
-      diff |= 0xff;
-
-    return diff + 1;
-  }
-
-  public String[] getLocations() throws IOException {
-    return locations;
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    range.readFields(in);
-    int numLocs = in.readInt();
-    locations = new String[numLocs];
-    for (int i = 0; i < numLocs; ++i)
-      locations[i] = in.readUTF();
-    
-    if (in.readBoolean()) {
-      isolatedScan = in.readBoolean();
-    }
-    
-    if (in.readBoolean()) {
-      offline = in.readBoolean();
-    }
-    
-    if (in.readBoolean()) {
-      localIterators = in.readBoolean();
-    }
-    
-    if (in.readBoolean()) {
-      mockInstance = in.readBoolean();
-    }
-    
-    if (in.readBoolean()) {
-      int numColumns = in.readInt();
-      List<String> columns = new ArrayList<String>(numColumns);
-      for (int i = 0; i < numColumns; i++) {
-        columns.add(in.readUTF());
-      }
-      
-      fetchedColumns = InputConfigurator.deserializeFetchedColumns(columns);
-    }
-    
-    if (in.readBoolean()) {
-      String strAuths = in.readUTF();
-      auths = new Authorizations(strAuths.getBytes(Charset.forName("UTF-8")));
-    }
-    
-    if (in.readBoolean()) {
-      principal = in.readUTF();
-    }
-    
-    if (in.readBoolean()) {
-      String tokenClass = in.readUTF();
-      byte[] base64TokenBytes = in.readUTF().getBytes(Charset.forName("UTF-8"));
-      byte[] tokenBytes = Base64.decodeBase64(base64TokenBytes);
-      
-      try {
-        token = CredentialHelper.extractToken(tokenClass, tokenBytes);
-      } catch (AccumuloSecurityException e) {
-        throw new IOException(e);
-      }
-    }
-    
-    if (in.readBoolean()) {
-      instanceName = in.readUTF();
-    }
-    
-    if (in.readBoolean()) {
-      zooKeepers = in.readUTF();
-    }
-    
-    if (in.readBoolean()) {
-      level = Level.toLevel(in.readInt());
-    }
-  }
-
-  public void write(DataOutput out) throws IOException {
-    range.write(out);
-    out.writeInt(locations.length);
-    for (int i = 0; i < locations.length; ++i)
-      out.writeUTF(locations[i]);
-    
-    out.writeBoolean(null != isolatedScan);
-    if (null != isolatedScan) {
-      out.writeBoolean(isolatedScan);
-    }
-    
-    out.writeBoolean(null != offline);
-    if (null != offline) {
-      out.writeBoolean(offline);
-    }
-    
-    out.writeBoolean(null != localIterators);
-    if (null != localIterators) {
-      out.writeBoolean(localIterators);
-    }
-    
-    out.writeBoolean(null != mockInstance);
-    if (null != mockInstance) {
-      out.writeBoolean(mockInstance);
-    }
-    
-    out.writeBoolean(null != fetchedColumns);
-    if (null != fetchedColumns) {
-      String[] cols = InputConfigurator.serializeColumns(fetchedColumns);
-      out.writeInt(cols.length);
-      for (String col : cols) {
-        out.writeUTF(col);
-      }
-    }
-    
-    out.writeBoolean(null != auths);
-    if (null != auths) {
-      out.writeUTF(auths.serialize());
-    }
-    
-    out.writeBoolean(null != principal);
-    if (null != principal) {
-      out.writeUTF(principal);
-    }
-    
-    out.writeBoolean(null != token);
-    if (null != token) {
-      out.writeUTF(token.getClass().getCanonicalName());
-      try {
-        out.writeUTF(CredentialHelper.tokenAsBase64(token));
-      } catch (AccumuloSecurityException e) {
-        throw new IOException(e);
-      }
-    }
-    
-    out.writeBoolean(null != instanceName);
-    if (null != instanceName) {
-      out.writeUTF(instanceName);
-    }
-    
-    out.writeBoolean(null != zooKeepers);
-    if (null != zooKeepers) {
-      out.writeUTF(zooKeepers);
-    }
-    
-    out.writeBoolean(null != level);
-    if (null != level) {
-      out.writeInt(level.toInt());
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(128);
-    sb.append("Range: ").append(range);
-    sb.append(" Locations: ").append(locations);
-    sb.append(" Table: ").append(table);
-    // TODO finish building of string
-    return sb.toString();
-  }
-
-  public String getTable() {
-    return table;
-  }
-
-  public void setTable(String table) {
-    this.table = table;
-  }
-  
-  public Instance getInstance() {
-    if (null == instanceName) {
-      return null;
-    }
-    
-    if (isMockInstance()) {  
-      return new MockInstance(getInstanceName());
-    }
-    
-    if (null == zooKeepers) {
-      return null;
-    }
-    
-    return new ZooKeeperInstance(getInstanceName(), getZooKeepers());
-  }
-
-  public String getInstanceName() {
-    return instanceName;
+    super(range, locations);
   }
 
-  public void setInstanceName(String instanceName) {
-    this.instanceName = instanceName;
-  }
-
-  public String getZooKeepers() {
-    return zooKeepers;
-  }
-
-  public void setZooKeepers(String zooKeepers) {
-    this.zooKeepers = zooKeepers;
-  }
-
-  public String getPrincipal() {
-    return principal;
-  }
-
-  public void setPrincipal(String principal) {
-    this.principal = principal;
-  }
-  
-  public AuthenticationToken getToken() {
-    return token;
-  }
-  
-  public void setToken(AuthenticationToken token) {
-    this.token = token;;
-  }
-
-  public Boolean isOffline() {
-    return offline;
-  }
-
-  public void setOffline(Boolean offline) {
-    this.offline = offline;
-  }
-
-  public void setLocations(String[] locations) {
-    this.locations = locations;
-  }
-
-  public Boolean isMockInstance() {
-    return mockInstance;
-  }
-
-  public void setMockInstance(Boolean mockInstance) {
-    this.mockInstance = mockInstance;
-  }
-
-  public Boolean isIsolatedScan() {
-    return isolatedScan;
-  }
-
-  public void setIsolatedScan(Boolean isolatedScan) {
-    this.isolatedScan = isolatedScan;
-  }
-
-  public Authorizations getAuths() {
-    return auths;
-  }
-
-  public void setAuths(Authorizations auths) {
-    this.auths = auths;
-  }
-
-  public void setRange(Range range) {
-    this.range = range;
-  }
-
-  public Boolean usesLocalIterators() {
-    return localIterators;
-  }
-
-  public void setUsesLocalIterators(Boolean localIterators) {
-    this.localIterators = localIterators;
-  }
-
-  public Set<Pair<Text,Text>> getFetchedColumns() {
-    return fetchedColumns;
-  }
-
-  public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
-    this.fetchedColumns = fetchedColumns;
-  }
-
-  public List<IteratorSetting> getIterators() {
-    return iterators;
-  }
-
-  public void setIterators(List<IteratorSetting> iterators) {
-    this.iterators = iterators;
-  }
-
-  public Level getLogLevel() {
-    return level;
-  }
-  
-  public void setLogLevel(Level level) {
-    this.level = level;
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/67189202/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index 9855c31..8ff3f0e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -279,11 +279,22 @@ public class RangeInputSplit extends InputSplit implements Writable {
 
   @Override
   public String toString() {
-    StringBuilder sb = new StringBuilder(128);
+    StringBuilder sb = new StringBuilder(256);
     sb.append("Range: ").append(range);
     sb.append(" Locations: ").append(locations);
     sb.append(" Table: ").append(table);
-    // TODO finish building of string
+    sb.append(" InstanceName: ").append(instanceName);
+    sb.append(" zooKeepers: ").append(zooKeepers);
+    sb.append(" principal: ").append(principal);
+    sb.append(" authenticationToken: ").append(token);
+    sb.append(" Authorizations: ").append(auths);
+    sb.append(" offlineScan: ").append(offline);
+    sb.append(" mockInstance: ").append(mockInstance);
+    sb.append(" isolatedScan: ").append(isolatedScan);
+    sb.append(" localIterators: ").append(localIterators);
+    sb.append(" fetchColumns: ").append(fetchedColumns);
+    sb.append(" iterators: ").append(iterators);
+    sb.append(" logLevel: ").append(level);
     return sb.toString();
   }
 


[11/19] git commit: ACCUMULO-1854 Account for 1.4 to 1.5 changes in how Authorizations.serialize works.

Posted by el...@apache.org.
ACCUMULO-1854 Account for 1.4 to 1.5 changes in how Authorizations.serialize works.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/ef64992c
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/ef64992c
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/ef64992c

Branch: refs/heads/1.5.1-SNAPSHOT
Commit: ef64992c6ef35ee4aba1ea0970981761e589e8c4
Parents: e08736d
Author: Josh Elser <el...@apache.org>
Authored: Thu Nov 21 21:39:51 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Nov 21 21:39:51 2013 -0500

----------------------------------------------------------------------
 .../apache/accumulo/core/client/mapreduce/RangeInputSplit.java  | 5 +++--
 .../accumulo/core/client/mapreduce/RangeInputSplitTest.java     | 2 --
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ef64992c/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index 6decdc6..9855c31 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -166,14 +166,15 @@ public class RangeInputSplit extends InputSplit implements Writable {
       int numColumns = in.readInt();
       List<String> columns = new ArrayList<String>(numColumns);
       for (int i = 0; i < numColumns; i++) {
-        columns.set(i, in.readUTF());
+        columns.add(in.readUTF());
       }
       
       fetchedColumns = InputConfigurator.deserializeFetchedColumns(columns);
     }
     
     if (in.readBoolean()) {
-      auths = new Authorizations(StringUtils.split(in.readUTF()));
+      String strAuths = in.readUTF();
+      auths = new Authorizations(strAuths.getBytes(Charset.forName("UTF-8")));
     }
     
     if (in.readBoolean()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ef64992c/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
index f6c604f..6da07a5 100644
--- 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
@@ -39,8 +39,6 @@ public class RangeInputSplitTest {
     Assert.assertTrue(Arrays.equals(split.getLocations(), newSplit.getLocations()));
   }
 
-
-
   @Test
   public void testAllFieldsWritable() throws IOException {
     RangeInputSplit split = new RangeInputSplit(new Range(new Key("a"), new Key("b")), new String[]{"localhost"});