You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2012/09/29 07:43:13 UTC

svn commit: r1391754 [8/11] - in /accumulo/branches/ACCUMULO-259: ./ assemble/ conf/ conf/examples/1GB/native-standalone/ conf/examples/1GB/standalone/ conf/examples/2GB/native-standalone/ conf/examples/2GB/standalone/ conf/examples/3GB/native-standalo...

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java Sat Sep 29 05:42:59 2012
@@ -19,23 +19,27 @@ package org.apache.accumulo.core.client.
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.HashSet;
 import java.util.List;
 
 import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.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.mock.MockInstance;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
 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.ContextFactory;
+import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -89,15 +93,19 @@ public class AccumuloInputFormatTest {
   
   /**
    * Check that the iterator configuration is getting stored in the Job conf correctly.
+   * @throws IOException 
    */
   @Test
-  public void testSetIterator() {
+  public void testSetIterator() throws IOException {
     JobContext job = ContextFactory.createJobContext();
     
-    AccumuloInputFormat.addIterator(job.getConfiguration(), new IteratorSetting(1, "WholeRow", "org.apache.accumulo.core.iterators.WholeRowIterator"));
+    IteratorSetting is = new IteratorSetting(1, "WholeRow", "org.apache.accumulo.core.iterators.WholeRowIterator");
+    AccumuloInputFormat.addIterator(job.getConfiguration(), is);
     Configuration conf = job.getConfiguration();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    is.write(new DataOutputStream(baos));
     String iterators = conf.get("AccumuloInputFormat.iterators");
-    assertEquals("1:org.apache.accumulo.core.iterators.WholeRowIterator:WholeRow", iterators);
+    assertEquals(new String(Base64.encodeBase64(baos.toByteArray())), iterators);
   }
   
   @Test
@@ -111,35 +119,31 @@ public class AccumuloInputFormatTest {
     iter.addOption("junk", "\0omg:!\\xyzzy");
     AccumuloInputFormat.addIterator(job.getConfiguration(), iter);
     
-    List<AccumuloIterator> list = AccumuloInputFormat.getIterators(job.getConfiguration());
+    List<IteratorSetting> 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);
+    IteratorSetting setting = list.get(0);
     assertEquals(1, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.user.WholeRowIterator", setting.getIteratorClass());
-    assertEquals("WholeRow", setting.getIteratorName());
+    assertEquals("WholeRow", setting.getName());
+    assertEquals(0, setting.getOptions().size());
     
     setting = list.get(1);
     assertEquals(2, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
-    assertEquals("Versions", setting.getIteratorName());
+    assertEquals("Versions", setting.getName());
+    assertEquals(0, setting.getOptions().size());
     
     setting = list.get(2);
     assertEquals(3, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
-    assertEquals("Count", setting.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());
+    assertEquals("Count", setting.getName());
+    assertEquals(2, setting.getOptions().size());
+    assertEquals("1", setting.getOptions().get("v1"));
+    assertEquals("\0omg:!\\xyzzy", setting.getOptions().get("junk"));
   }
   
   /**
@@ -157,25 +161,22 @@ public class AccumuloInputFormatTest {
     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);
+    List<IteratorSetting> list = AccumuloInputFormat.getIterators(job.getConfiguration());
+    assertEquals(1, list.size());
+    assertEquals(1, list.get(0).getOptions().size());
+    assertEquals(list.get(0).getOptions().get(key), value);
     
     someSetting.addOption(key + "2", value);
     someSetting.setPriority(2);
     someSetting.setName("it2");
     AccumuloInputFormat.addIterator(job.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);
-    }
+    list = AccumuloInputFormat.getIterators(job.getConfiguration());
+    assertEquals(2, list.size());
+    assertEquals(1, list.get(0).getOptions().size());
+    assertEquals(list.get(0).getOptions().get(key), value);
+    assertEquals(2, list.get(1).getOptions().size());
+    assertEquals(list.get(1).getOptions().get(key), value);
+    assertEquals(list.get(1).getOptions().get(key + "2"), value);
   }
   
   /**
@@ -189,26 +190,26 @@ public class AccumuloInputFormatTest {
     AccumuloInputFormat.addIterator(job.getConfiguration(), new IteratorSetting(2, "Versions", "org.apache.accumulo.core.iterators.VersioningIterator"));
     AccumuloInputFormat.addIterator(job.getConfiguration(), new IteratorSetting(3, "Count", "org.apache.accumulo.core.iterators.CountingIterator"));
     
-    List<AccumuloIterator> list = AccumuloInputFormat.getIterators(job.getConfiguration());
+    List<IteratorSetting> 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);
+    IteratorSetting setting = list.get(0);
     assertEquals(1, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.WholeRowIterator", setting.getIteratorClass());
-    assertEquals("WholeRow", setting.getIteratorName());
+    assertEquals("WholeRow", setting.getName());
     
     setting = list.get(1);
     assertEquals(2, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
-    assertEquals("Versions", setting.getIteratorName());
+    assertEquals("Versions", setting.getName());
     
     setting = list.get(2);
     assertEquals(3, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
-    assertEquals("Count", setting.getIteratorName());
+    assertEquals("Count", setting.getName());
     
   }
   
@@ -222,19 +223,22 @@ public class AccumuloInputFormatTest {
     RegExFilter.setRegexs(is, regex, null, null, null, false);
     AccumuloInputFormat.addIterator(job.getConfiguration(), is);
     
-    assertTrue(regex.equals(AccumuloInputFormat.getIterators(job.getConfiguration()).get(0).getIteratorName()));
+    assertTrue(regex.equals(AccumuloInputFormat.getIterators(job.getConfiguration()).get(0).getName()));
   }
   
   static class TestMapper extends Mapper<Key,Value,Key,Value> {
     Key key = null;
+    int first = 0;
     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));
+      else
+        first = Integer.parseInt(k.getRow().toString(), 16) - 1;
+      assertEquals(k.getRow(), new Text(String.format("%09x", first + count + 1)));
+      assertEquals(new String(v.get()), String.format("%09x", first + count));
       key = new Key(k);
       count++;
     }
@@ -245,7 +249,7 @@ public class AccumuloInputFormatTest {
     MockInstance mockInstance = new MockInstance("testmapinstance");
     Connector c = mockInstance.getConnector("root", new byte[] {});
     c.tableOperations().create("testtable");
-    BatchWriter bw = c.createBatchWriter("testtable", 10000L, 1000L, 4);
+    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()));
@@ -259,10 +263,14 @@ public class AccumuloInputFormatTest {
     job.setNumReduceTasks(0);
     AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable", new Authorizations());
     AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
+    HashSet<Range> ranges = new HashSet<Range>();
+    ranges.add(new Range("000000000", "000000010"));
+    ranges.add(new Range("000000100", "000000110"));
+    AccumuloInputFormat.setRanges(job.getConfiguration(), ranges);
     
     AccumuloInputFormat input = new AccumuloInputFormat();
     List<InputSplit> splits = input.getSplits(job);
-    assertEquals(splits.size(), 1);
+    assertEquals(splits.size(), 2);
     
     TestMapper mapper = (TestMapper) job.getMapperClass().newInstance();
     for (InputSplit split : splits) {
@@ -271,6 +279,7 @@ public class AccumuloInputFormatTest {
       Mapper<Key,Value,Key,Value>.Context context = ContextFactory.createMapContext(mapper, tac, reader, null, split);
       reader.initialize(split, context);
       mapper.run(context);
+      assertEquals(mapper.count, 16);
     }
   }
   
@@ -279,7 +288,7 @@ public class AccumuloInputFormatTest {
     MockInstance mockInstance = new MockInstance("testmapinstance");
     Connector c = mockInstance.getConnector("root", new byte[] {});
     c.tableOperations().create("testtable2");
-    BatchWriter bw = c.createBatchWriter("testtable2", 10000L, 1000L, 4);
+    BatchWriter bw = c.createBatchWriter("testtable2", 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()));
@@ -291,7 +300,9 @@ public class AccumuloInputFormatTest {
     AccumuloInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "testtable2", new Authorizations());
     AccumuloInputFormat.setMockInstance(job.getConfiguration(), "testmapinstance");
     AccumuloInputFormat input = new AccumuloInputFormat();
-    RangeInputSplit ris = new RangeInputSplit();
+    List<InputSplit> splits = input.getSplits(job);
+    assertEquals(splits.size(), 1);
+    RangeInputSplit ris = (RangeInputSplit) splits.get(0);
     TaskAttemptContext tac = ContextFactory.createTaskAttemptContext(job);
     RecordReader<Key,Value> rr = input.createRecordReader(ris, tac);
     rr.initialize(ris, tac);
@@ -302,5 +313,6 @@ public class AccumuloInputFormatTest {
     while (rr.nextKeyValue()) {
       mapper.map(rr.getCurrentKey(), rr.getCurrentValue(), (TestMapper.Context) context);
     }
+    assertEquals(mapper.count, 100);
   }
 }

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java Sat Sep 29 05:42:59 2012
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.mock.MockInstance;
@@ -78,7 +79,7 @@ public class AccumuloOutputFormatTest {
     Connector c = mockInstance.getConnector("root", new byte[] {});
     c.tableOperations().create("testtable1");
     c.tableOperations().create("testtable2");
-    BatchWriter bw = c.createBatchWriter("testtable1", 10000L, 1000L, 4);
+    BatchWriter bw = c.createBatchWriter("testtable1", 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()));

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java Sat Sep 29 05:42:59 2012
@@ -20,13 +20,20 @@ import static org.junit.Assert.assertEqu
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.data.Key;
@@ -44,78 +51,87 @@ import org.apache.hadoop.mapreduce.TaskA
 import org.junit.Test;
 
 public class AccumuloRowInputFormatTest {
-  List<Entry<Key,Value>> row1;
-  List<Entry<Key,Value>> row2;
-  List<Entry<Key,Value>> row3;
+  private static final String ROW1 = "row1";
+  private static final String ROW2 = "row2";
+  private static final String ROW3 = "row3";
+  private static final String COLF1 = "colf1";
+  private transient final List<Entry<Key,Value>> row1;
+  private transient final List<Entry<Key,Value>> row2;
+  private transient final List<Entry<Key,Value>> row3;
   
-  {
+  public AccumuloRowInputFormatTest() {
     row1 = new ArrayList<Entry<Key,Value>>();
-    row1.add(new KeyValue(new Key("row1", "colf1", "colq1"), "v1".getBytes()));
-    row1.add(new KeyValue(new Key("row1", "colf1", "colq2"), "v2".getBytes()));
-    row1.add(new KeyValue(new Key("row1", "colf2", "colq3"), "v3".getBytes()));
+    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq1"), "v1".getBytes()));
+    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq2"), "v2".getBytes()));
+    row1.add(new KeyValue(new Key(ROW1, "colf2", "colq3"), "v3".getBytes()));
     row2 = new ArrayList<Entry<Key,Value>>();
-    row2.add(new KeyValue(new Key("row2", "colf1", "colq4"), "v4".getBytes()));
+    row2.add(new KeyValue(new Key(ROW2, COLF1, "colq4"), "v4".getBytes()));
     row3 = new ArrayList<Entry<Key,Value>>();
-    row3.add(new KeyValue(new Key("row3", "colf1", "colq5"), "v5".getBytes()));
+    row3.add(new KeyValue(new Key(ROW3, COLF1, "colq5"), "v5".getBytes()));
   }
-  
-  public static void checkLists(List<Entry<Key,Value>> a, List<Entry<Key,Value>> b) {
-    assertEquals(a.size(), b.size());
-    for (int i = 0; i < a.size(); i++) {
-      assertEquals(a.get(i).getKey(), b.get(i).getKey());
-      assertEquals(a.get(i).getValue(), b.get(i).getValue());
+
+  public static void checkLists(final List<Entry<Key,Value>> first, final List<Entry<Key,Value>> second) {
+    assertEquals("Sizes should be the same.", first.size(), second.size());
+    for (int i = 0; i < first.size(); i++) {
+      assertEquals("Keys should be equal.", first.get(i).getKey(), second.get(i).getKey());
+      assertEquals("Values should be equal.", first.get(i).getValue(), second.get(i).getValue());
     }
   }
   
-  public static void checkLists(List<Entry<Key,Value>> a, Iterator<Entry<Key,Value>> b) {
-    int i = 0;
-    while (b.hasNext()) {
-      Entry<Key,Value> e = b.next();
-      assertEquals(a.get(i).getKey(), e.getKey());
-      assertEquals(a.get(i).getValue(), e.getValue());
-      i++;
+  public static void checkLists(final List<Entry<Key,Value>> first, final Iterator<Entry<Key,Value>> second) {
+    int entryIndex = 0; // NOPMD
+    while (second.hasNext()) {
+      final Entry<Key,Value> entry = second.next();
+      assertEquals("Keys should be equal", first.get(entryIndex).getKey(), entry.getKey());
+      assertEquals("Values should be equal", first.get(entryIndex).getValue(), entry.getValue());
+      entryIndex++; // NOPMD
     }
   }
   
-  public static void insertList(BatchWriter bw, List<Entry<Key,Value>> list) throws Exception {
+  public static void insertList(final BatchWriter writer, final List<Entry<Key,Value>> list) throws MutationsRejectedException {
     for (Entry<Key,Value> e : list) {
-      Key k = e.getKey();
-      Mutation m = new Mutation(k.getRow());
-      m.put(k.getColumnFamily(), k.getColumnQualifier(), new ColumnVisibility(k.getColumnVisibility()), k.getTimestamp(), e.getValue());
-      bw.addMutation(m);
+      final Key key = e.getKey();
+      final Mutation mutation = new Mutation(key.getRow());  // NOPMD
+      ColumnVisibility colVisibility = new ColumnVisibility(key.getColumnVisibility()); // NOPMD
+      mutation.put(key.getColumnFamily(), key.getColumnQualifier(), colVisibility, key.getTimestamp(), e.getValue());
+      writer.addMutation(mutation);
     }
   }
   
   @Test
-  public void test() throws Exception {
-    MockInstance instance = new MockInstance("instance1");
-    Connector conn = instance.getConnector("root", "".getBytes());
+  public void test() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException, IOException, InterruptedException {
+    final MockInstance instance = new MockInstance("instance1");
+    final Connector conn = instance.getConnector("root", "".getBytes());
     conn.tableOperations().create("test");
-    BatchWriter bw = conn.createBatchWriter("test", 100000l, 100l, 5);
-    
-    insertList(bw, row1);
-    insertList(bw, row2);
-    insertList(bw, row3);
-    bw.close();
-    
-    JobContext job = ContextFactory.createJobContext();
+    BatchWriter writer = null; // NOPMD
+    try {
+      writer = conn.createBatchWriter("test", new BatchWriterConfig());
+        insertList(writer, row1);
+        insertList(writer, row2);
+        insertList(writer, row3);
+    } finally {
+      if (writer != null) {
+    	  writer.close();
+      }
+    }
+    final JobContext job = ContextFactory.createJobContext();
     AccumuloRowInputFormat.setInputInfo(job.getConfiguration(), "root", "".getBytes(), "test", new Authorizations());
     AccumuloRowInputFormat.setMockInstance(job.getConfiguration(), "instance1");
-    AccumuloRowInputFormat crif = new AccumuloRowInputFormat();
-    RangeInputSplit ris = new RangeInputSplit();
-    TaskAttemptContext tac = ContextFactory.createTaskAttemptContext(job);
-    RecordReader<Text,PeekingIterator<Entry<Key,Value>>> rr = crif.createRecordReader(ris, tac);
-    rr.initialize(ris, tac);
+    final AccumuloRowInputFormat crif = new AccumuloRowInputFormat();
+    final RangeInputSplit ris = new RangeInputSplit();
+    final TaskAttemptContext tac = ContextFactory.createTaskAttemptContext(job);
+    final RecordReader<Text,PeekingIterator<Entry<Key,Value>>> recReader = crif.createRecordReader(ris, tac);
+    recReader.initialize(ris, tac);
     
-    assertTrue(rr.nextKeyValue());
-    assertEquals(new Text("row1"), rr.getCurrentKey());
-    checkLists(row1, rr.getCurrentValue());
-    assertTrue(rr.nextKeyValue());
-    assertEquals(new Text("row2"), rr.getCurrentKey());
-    checkLists(row2, rr.getCurrentValue());
-    assertTrue(rr.nextKeyValue());
-    assertEquals(new Text("row3"), rr.getCurrentKey());
-    checkLists(row3, rr.getCurrentValue());
-    assertFalse(rr.nextKeyValue());
+    assertTrue("Next key value should be true.", recReader.nextKeyValue());
+    assertEquals("Current key should be " + ROW1, new Text(ROW1), recReader.getCurrentKey());
+    checkLists(row1, recReader.getCurrentValue());
+    assertTrue("Next key value should be true.", recReader.nextKeyValue());
+    assertEquals("Current key should be " + ROW2, new Text(ROW2), recReader.getCurrentKey());
+    checkLists(row2, recReader.getCurrentValue());
+    assertTrue("Next key value should be true.", recReader.nextKeyValue());
+    assertEquals("Current key should be " + ROW3, new Text(ROW3), recReader.getCurrentKey());
+    checkLists(row3, recReader.getCurrentValue());
+    assertFalse("Next key value should be false.", recReader.nextKeyValue());
   }
 }

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java Sat Sep 29 05:42:59 2012
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.Constant
 import org.apache.accumulo.core.client.BatchDeleter;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
@@ -54,7 +55,7 @@ public class MockConnectorTest {
   public void testSunnyDay() throws Exception {
     Connector c = new MockConnector("root");
     c.tableOperations().create("test");
-    BatchWriter bw = c.createBatchWriter("test", 10000L, 1000L, 4);
+    BatchWriter bw = c.createBatchWriter("test", new BatchWriterConfig());
     for (int i = 0; i < 100; i++) {
       int r = random.nextInt();
       Mutation m = new Mutation(asText(r));
@@ -98,7 +99,7 @@ public class MockConnectorTest {
     c.tableOperations().attachIterator(table, is);
     String keys[][] = { {"foo", "day", "20080101"}, {"foo", "day", "20080101"}, {"foo", "day", "20080103"}, {"bar", "day", "20080101"},
         {"bar", "day", "20080101"},};
-    BatchWriter bw = c.createBatchWriter("perDayCounts", 1000L, 1000L, 1);
+    BatchWriter bw = c.createBatchWriter("perDayCounts", new BatchWriterConfig());
     for (String elt[] : keys) {
       Mutation m = new Mutation(new Text(elt[0]));
       m.put(new Text(elt[1]), new Text(elt[2]), new Value("1".getBytes()));
@@ -121,7 +122,7 @@ public class MockConnectorTest {
   public void testDelete() throws Exception {
     Connector c = new MockConnector("root");
     c.tableOperations().create("test");
-    BatchWriter bw = c.createBatchWriter("test", 10000L, 1000L, 4);
+    BatchWriter bw = c.createBatchWriter("test", new BatchWriterConfig());
     
     Mutation m1 = new Mutation("r1");
     
@@ -165,14 +166,14 @@ public class MockConnectorTest {
       c.tableOperations().delete("test");
     c.tableOperations().create("test");
     
-    BatchDeleter deleter = c.createBatchDeleter("test", Constants.NO_AUTHS, 2, 10000L, 1000L, 4);
+    BatchDeleter deleter = c.createBatchDeleter("test", Constants.NO_AUTHS, 2, new BatchWriterConfig());
     // first make sure it deletes fine when its empty
     deleter.setRanges(Collections.singletonList(new Range(("r1"))));
     deleter.delete();
     this.checkRemaining(c, "test", 0);
     
     // test deleting just one row
-    BatchWriter writer = c.createBatchWriter("test", 10, 10, 1);
+    BatchWriter writer = c.createBatchWriter("test", new BatchWriterConfig());
     Mutation m = new Mutation("r1");
     m.put("fam", "qual", "value");
     writer.addMutation(m);
@@ -186,7 +187,7 @@ public class MockConnectorTest {
     this.checkRemaining(c, "test", 0);
     
     // test multi row deletes
-    writer = c.createBatchWriter("test", 10, 10, 1);
+    writer = c.createBatchWriter("test", new BatchWriterConfig());
     m = new Mutation("r1");
     m.put("fam", "qual", "value");
     writer.addMutation(m);
@@ -231,7 +232,7 @@ public class MockConnectorTest {
     // test writing to a table that the is being scanned
     Connector c = new MockConnector("root");
     c.tableOperations().create("test");
-    BatchWriter bw = c.createBatchWriter("test", 10000L, 1000L, 4);
+    BatchWriter bw = c.createBatchWriter("test", new BatchWriterConfig());
     
     for (int i = 0; i < 10; i++) {
       Mutation m1 = new Mutation("r" + i);
@@ -283,7 +284,7 @@ public class MockConnectorTest {
     Connector c = new MockConnector("root");
     c.tableOperations().create("a");
     c.tableOperations().create("b");
-    MultiTableBatchWriter bw = c.createMultiTableBatchWriter(10000L, 1000L, 4);
+    MultiTableBatchWriter bw = c.createMultiTableBatchWriter(new BatchWriterConfig());
     Mutation m1 = new Mutation("r1");
     m1.put("cf1", "cq1", 1, "v1");
     BatchWriter b = bw.getBatchWriter("a");
@@ -314,7 +315,7 @@ public class MockConnectorTest {
   public void testUpdate() throws Exception {
     Connector c = new MockConnector("root");
     c.tableOperations().create("test");
-    BatchWriter bw = c.createBatchWriter("test", 1000, 1000l, 1);
+    BatchWriter bw = c.createBatchWriter("test", new BatchWriterConfig());
     
     for (int i = 0; i < 10; i++) {
       Mutation m = new Mutation("r1");

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/data/KeyTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/data/KeyTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/data/KeyTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/data/KeyTest.java Sat Sep 29 05:42:59 2012
@@ -27,8 +27,8 @@ public class KeyTest extends TestCase {
     Key k3 = new Key("r1".getBytes(), "cf".getBytes(), "cq".getBytes(), new byte[0], 0, true);
     Key k4 = new Key("r1".getBytes(), "cf".getBytes(), "cq".getBytes(), new byte[0], 0, true);
     
-    assertTrue(k1.compareTo(k2) == 0);
-    assertTrue(k3.compareTo(k4) == 0);
+    assertTrue(k1.equals(k2));
+    assertTrue(k3.equals(k4));
     assertTrue(k1.compareTo(k3) > 0);
     assertTrue(k3.compareTo(k1) < 0);
   }

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java Sat Sep 29 05:42:59 2012
@@ -138,7 +138,7 @@ public class RFileTest extends TestCase 
         
       }
       
-      if (reader.getLastKey().compareTo(lastKey) != 0) {
+      if (!reader.getLastKey().equals(lastKey)) {
         throw new RuntimeException("Last key out of order " + reader.getLastKey() + " " + lastKey);
       }
     }

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java Sat Sep 29 05:42:59 2012
@@ -65,9 +65,7 @@ public class IndexedDocIteratorTest exte
     docColf.append("type".getBytes(), 0, "type".getBytes().length);
   }
   
-  static float hitRatio = 0.1f;
-  
-  private synchronized static TreeMap<Key,Value> createSortedMap(int numRows, int numDocsPerRow, Text[] columnFamilies, Text[] otherColumnFamilies,
+  private TreeMap<Key,Value> createSortedMap(float hitRatio, int numRows, int numDocsPerRow, Text[] columnFamilies, Text[] otherColumnFamilies,
       HashSet<Text> docs, Text[] negatedColumns) {
     StringBuilder sb = new StringBuilder();
     Random r = new Random();
@@ -135,18 +133,18 @@ public class IndexedDocIteratorTest exte
   
   static TestRFile trf = new TestRFile();
   
-  private synchronized static SortedKeyValueIterator<Key,Value> createIteratorStack(int numRows, int numDocsPerRow, Text[] columnFamilies,
+  private SortedKeyValueIterator<Key,Value> createIteratorStack(float hitRatio, int numRows, int numDocsPerRow, Text[] columnFamilies,
       Text[] otherColumnFamilies, HashSet<Text> docs) throws IOException {
     Text nullText[] = new Text[0];
-    return createIteratorStack(numRows, numDocsPerRow, columnFamilies, otherColumnFamilies, docs, nullText);
+    return createIteratorStack(hitRatio, numRows, numDocsPerRow, columnFamilies, otherColumnFamilies, docs, nullText);
   }
   
-  private synchronized static SortedKeyValueIterator<Key,Value> createIteratorStack(int numRows, int numDocsPerRow, Text[] columnFamilies,
+  private SortedKeyValueIterator<Key,Value> createIteratorStack(float hitRatio, int numRows, int numDocsPerRow, Text[] columnFamilies,
       Text[] otherColumnFamilies, HashSet<Text> docs, Text[] negatedColumns) throws IOException {
     // write a map file
     trf.openWriter(false);
     
-    TreeMap<Key,Value> inMemoryMap = createSortedMap(numRows, numDocsPerRow, columnFamilies, otherColumnFamilies, docs, negatedColumns);
+    TreeMap<Key,Value> inMemoryMap = createSortedMap(hitRatio, numRows, numDocsPerRow, columnFamilies, otherColumnFamilies, docs, negatedColumns);
     trf.writer.startNewLocalityGroup("docs", RFileTest.ncfs(docColf.toString()));
     for (Entry<Key,Value> entry : inMemoryMap.entrySet()) {
       if (entry.getKey().getColumnFamily().equals(docColf))
@@ -189,9 +187,9 @@ public class IndexedDocIteratorTest exte
     otherColumnFamilies[2] = new Text("D");
     otherColumnFamilies[3] = new Text("F");
     
-    hitRatio = 0.5f;
+    float hitRatio = 0.5f;
     HashSet<Text> docs = new HashSet<Text>();
-    SortedKeyValueIterator<Key,Value> source = createIteratorStack(NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
+    SortedKeyValueIterator<Key,Value> source = createIteratorStack(hitRatio, NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
     IteratorSetting is = new IteratorSetting(1, IndexedDocIterator.class);
     IndexedDocIterator.setColumnFamilies(is, columnFamilies);
     IndexedDocIterator.setColfs(is, indexColf.toString(), docColfPrefix);
@@ -227,9 +225,9 @@ public class IndexedDocIteratorTest exte
     otherColumnFamilies[2] = new Text("D");
     otherColumnFamilies[3] = new Text("F");
     
-    hitRatio = 0.5f;
+    float hitRatio = 0.5f;
     HashSet<Text> docs = new HashSet<Text>();
-    SortedKeyValueIterator<Key,Value> source = createIteratorStack(NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
+    SortedKeyValueIterator<Key,Value> source = createIteratorStack(hitRatio, NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
     IteratorSetting is = new IteratorSetting(1, IndexedDocIterator.class);
     IndexedDocIterator.setColumnFamilies(is, columnFamilies);
     IndexedDocIterator.setColfs(is, indexColf.toString(), docColfPrefix);
@@ -264,10 +262,10 @@ public class IndexedDocIteratorTest exte
     otherColumnFamilies[2] = new Text("D");
     otherColumnFamilies[3] = new Text("F");
     
-    hitRatio = 0.5f;
+    float hitRatio = 0.5f;
     HashSet<Text> docs = new HashSet<Text>();
-    SortedKeyValueIterator<Key,Value> source = createIteratorStack(NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
-    SortedKeyValueIterator<Key,Value> source2 = createIteratorStack(NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
+    SortedKeyValueIterator<Key,Value> source = createIteratorStack(hitRatio, NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
+    SortedKeyValueIterator<Key,Value> source2 = createIteratorStack(hitRatio, NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
     ArrayList<SortedKeyValueIterator<Key,Value>> sourceIters = new ArrayList<SortedKeyValueIterator<Key,Value>>();
     sourceIters.add(source);
     sourceIters.add(source2);
@@ -310,9 +308,9 @@ public class IndexedDocIteratorTest exte
     otherColumnFamilies[2] = new Text("D");
     otherColumnFamilies[3] = new Text("F");
     
-    hitRatio = 0.5f;
+    float hitRatio = 0.5f;
     HashSet<Text> docs = new HashSet<Text>();
-    SortedKeyValueIterator<Key,Value> source = createIteratorStack(NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs, negatedColumns);
+    SortedKeyValueIterator<Key,Value> source = createIteratorStack(hitRatio, NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs, negatedColumns);
     IteratorSetting is = new IteratorSetting(1, IndexedDocIterator.class);
     IndexedDocIterator.setColumnFamilies(is, columnFamilies, notFlags);
     IndexedDocIterator.setColfs(is, indexColf.toString(), docColfPrefix);

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java Sat Sep 29 05:42:59 2012
@@ -32,6 +32,7 @@ import junit.framework.TestCase;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.mock.MockInstance;
@@ -63,15 +64,13 @@ public class IntersectingIteratorTest ex
   Text[] searchFamilies;
   boolean[] notFlags;
   
-  static int docid = 0;
+  int docid = 0;
   
   static {
     log.setLevel(Level.OFF);
   }
   
-  static float hitRatio = 0.1f;
-  
-  private synchronized static TreeMap<Key,Value> createSortedMap(int numRows, int numDocsPerRow, Text[] columnFamilies, Text[] otherColumnFamilies,
+  private TreeMap<Key,Value> createSortedMap(float hitRatio, int numRows, int numDocsPerRow, Text[] columnFamilies, Text[] otherColumnFamilies,
       HashSet<Text> docs, Text[] negatedColumns) {
     Random r = new Random();
     Value v = new Value(new byte[0]);
@@ -115,19 +114,19 @@ public class IntersectingIteratorTest ex
     return map;
   }
   
-  private synchronized static SortedKeyValueIterator<Key,Value> createIteratorStack(int numRows, int numDocsPerRow, Text[] columnFamilies,
+  private SortedKeyValueIterator<Key,Value> createIteratorStack(float hitRatio, int numRows, int numDocsPerRow, Text[] columnFamilies,
       Text[] otherColumnFamilies, HashSet<Text> docs) throws IOException {
     Text nullText[] = new Text[0];
-    return createIteratorStack(numRows, numDocsPerRow, columnFamilies, otherColumnFamilies, docs, nullText);
+    return createIteratorStack(hitRatio, numRows, numDocsPerRow, columnFamilies, otherColumnFamilies, docs, nullText);
   }
   
-  private synchronized static SortedKeyValueIterator<Key,Value> createIteratorStack(int numRows, int numDocsPerRow, Text[] columnFamilies,
+  private SortedKeyValueIterator<Key,Value> createIteratorStack(float hitRatio, int numRows, int numDocsPerRow, Text[] columnFamilies,
       Text[] otherColumnFamilies, HashSet<Text> docs, Text[] negatedColumns) throws IOException {
-    TreeMap<Key,Value> inMemoryMap = createSortedMap(numRows, numDocsPerRow, columnFamilies, otherColumnFamilies, docs, negatedColumns);
+    TreeMap<Key,Value> inMemoryMap = createSortedMap(hitRatio, numRows, numDocsPerRow, columnFamilies, otherColumnFamilies, docs, negatedColumns);
     return new SortedMapIterator(inMemoryMap);
   }
   
-  private synchronized static void cleanup() throws IOException {
+  private void cleanup() throws IOException {
     docid = 0;
   }
   
@@ -151,8 +150,8 @@ public class IntersectingIteratorTest ex
     otherColumnFamilies[2] = new Text("D");
     otherColumnFamilies[3] = new Text("F");
     
-    hitRatio = 0.5f;
-    SortedKeyValueIterator<Key,Value> source = createIteratorStack(NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
+    float hitRatio = 0.5f;
+    SortedKeyValueIterator<Key,Value> source = createIteratorStack(hitRatio, NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
     IteratorSetting is = new IteratorSetting(1, IntersectingIterator.class);
     IntersectingIterator.setColumnFamilies(is, columnFamilies);
     IntersectingIterator iter = new IntersectingIterator();
@@ -180,8 +179,8 @@ public class IntersectingIteratorTest ex
     otherColumnFamilies[2] = new Text("D");
     otherColumnFamilies[3] = new Text("F");
     
-    hitRatio = 0.5f;
-    SortedKeyValueIterator<Key,Value> source = createIteratorStack(NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
+    float hitRatio = 0.5f;
+    SortedKeyValueIterator<Key,Value> source = createIteratorStack(hitRatio, NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
     IteratorSetting is = new IteratorSetting(1, IntersectingIterator.class);
     IntersectingIterator.setColumnFamilies(is, columnFamilies);
     IntersectingIterator iter = new IntersectingIterator();
@@ -212,9 +211,9 @@ public class IntersectingIteratorTest ex
     otherColumnFamilies[2] = new Text("D");
     otherColumnFamilies[3] = new Text("F");
     
-    hitRatio = 0.5f;
-    SortedKeyValueIterator<Key,Value> source = createIteratorStack(NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
-    SortedKeyValueIterator<Key,Value> source2 = createIteratorStack(NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
+    float hitRatio = 0.5f;
+    SortedKeyValueIterator<Key,Value> source = createIteratorStack(hitRatio, NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
+    SortedKeyValueIterator<Key,Value> source2 = createIteratorStack(hitRatio, NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs);
     ArrayList<SortedKeyValueIterator<Key,Value>> sourceIters = new ArrayList<SortedKeyValueIterator<Key,Value>>();
     sourceIters.add(source);
     sourceIters.add(source2);
@@ -253,8 +252,8 @@ public class IntersectingIteratorTest ex
     otherColumnFamilies[2] = new Text("D");
     otherColumnFamilies[3] = new Text("F");
     
-    hitRatio = 0.5f;
-    SortedKeyValueIterator<Key,Value> source = createIteratorStack(NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs, negatedColumns);
+    float hitRatio = 0.5f;
+    SortedKeyValueIterator<Key,Value> source = createIteratorStack(hitRatio, NUM_ROWS, NUM_DOCIDS, columnFamilies, otherColumnFamilies, docs, negatedColumns);
     IteratorSetting is = new IteratorSetting(1, IntersectingIterator.class);
     IntersectingIterator.setColumnFamilies(is, columnFamilies, notFlags);
     IntersectingIterator iter = new IntersectingIterator();
@@ -276,7 +275,7 @@ public class IntersectingIteratorTest ex
     MockInstance inst = new MockInstance("mockabye");
     Connector connector = inst.getConnector("user", "pass");
     connector.tableOperations().create("index");
-    BatchWriter bw = connector.createBatchWriter("index", 1000, 1000, 1);
+    BatchWriter bw = connector.createBatchWriter("index", new BatchWriterConfig());
     Mutation m = new Mutation("000012");
     m.put("rvy", "5000000000000000", empty);
     m.put("15qh", "5000000000000000", empty);

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java Sat Sep 29 05:42:59 2012
@@ -25,6 +25,7 @@ import junit.framework.TestCase;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
@@ -84,7 +85,7 @@ public class RowFilterTest extends TestC
     Connector conn = instance.getConnector("", "".getBytes());
     
     conn.tableOperations().create("table1");
-    BatchWriter bw = conn.createBatchWriter("table1", 1000000, 60000, 1);
+    BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
     
     Mutation m = new Mutation("0");
     m.put("cf1", "cq1", "1");

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/AuthorizationsTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/AuthorizationsTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/AuthorizationsTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/AuthorizationsTest.java Sat Sep 29 05:42:59 2012
@@ -31,10 +31,16 @@ public class AuthorizationsTest {
   
   @Test
   public void testEncodeDecode() {
-    Authorizations a = new Authorizations("a", "abcdefg", "hijklmno");
+    Authorizations a = new Authorizations("a", "abcdefg", "hijklmno", ",");
     byte[] array = a.getAuthorizationsArray();
     Authorizations b = new Authorizations(array);
     assertEquals(a, b);
+    
+    // test encoding empty auths
+    a = new Authorizations();
+    array = a.getAuthorizationsArray();
+    b = new Authorizations(array);
+    assertEquals(a, b);
   }
   
 }

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java Sat Sep 29 05:42:59 2012
@@ -108,4 +108,24 @@ public class ColumnVisibilityTest {
     shouldThrow("(A&B)|(C&D)&(E)");
     shouldThrow("a|b&c", "A&B&C|D", "(A&B)|(C&D)&(E)");
   }
+  
+  @Test
+  public void testQuotes() {
+    shouldThrow("\"\"");
+    shouldThrow("\"A\"A");
+    shouldThrow("\"A\"\"B\"");
+    shouldThrow("(A)\"B\"");
+    shouldThrow("\"A\"(B)");
+    shouldThrow("\"A");
+    shouldThrow("\"");
+    shouldThrow("\"B");
+    shouldThrow("A&\"B");
+    shouldThrow("A&\"B\\'");
+    
+    shouldNotThrow("\"A\"");
+    shouldNotThrow("(\"A\")");
+    shouldNotThrow("A&\"B.D\"");
+    shouldNotThrow("A&\"B\\\\D\"");
+    shouldNotThrow("A&\"B\\\"D\"");
+  }
 }

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/VisibilityEvaluatorTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/VisibilityEvaluatorTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/VisibilityEvaluatorTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/security/VisibilityEvaluatorTest.java Sat Sep 29 05:42:59 2012
@@ -16,10 +16,16 @@
  */
 package org.apache.accumulo.core.security;
 
+import static org.apache.accumulo.core.security.ColumnVisibility.quote;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.Charset;
+
+import org.apache.accumulo.core.util.BadArgumentException;
 import org.apache.accumulo.core.util.ByteArraySet;
 import org.junit.Test;
 
@@ -54,7 +60,7 @@ public class VisibilityEvaluatorTest {
       try {
         ct.evaluate(new ColumnVisibility(marking));
         fail(marking + " failed to throw");
-      } catch (Throwable e) {
+      } catch (BadArgumentException e) {
         // all is good
       }
     }
@@ -64,7 +70,7 @@ public class VisibilityEvaluatorTest {
       try {
         ct.evaluate(new ColumnVisibility(marking));
         fail(marking + " failed to throw");
-      } catch (Throwable e) {
+      } catch (BadArgumentException e) {
         // all is good
       }
     }
@@ -74,9 +80,47 @@ public class VisibilityEvaluatorTest {
       try {
         ct.evaluate(new ColumnVisibility(marking));
         fail(marking + " failed to throw");
-      } catch (Throwable e) {
+      } catch (BadArgumentException e) {
         // all is good
       }
     }
   }
+  
+  @Test
+  public void testQuotedExpressions() throws VisibilityParseException {
+    VisibilityEvaluator ct = new VisibilityEvaluator(new Authorizations("A#C", "A\"C", "A\\C", "AC"));
+    
+    assertTrue(ct.evaluate(new ColumnVisibility(quote("A#C") + "|" + quote("A?C"))));
+    assertTrue(ct.evaluate(new ColumnVisibility(new ColumnVisibility(quote("A#C") + "|" + quote("A?C")).flatten())));
+    assertTrue(ct.evaluate(new ColumnVisibility(quote("A\"C") + "&" + quote("A\\C"))));
+    assertTrue(ct.evaluate(new ColumnVisibility(new ColumnVisibility(quote("A\"C") + "&" + quote("A\\C")).flatten())));
+    assertTrue(ct.evaluate(new ColumnVisibility("(" + quote("A\"C") + "|B)&(" + quote("A#C") + "|D)")));
+    
+    assertFalse(ct.evaluate(new ColumnVisibility(quote("A#C") + "&B")));
+    
+    assertTrue(ct.evaluate(new ColumnVisibility(quote("A#C"))));
+    assertTrue(ct.evaluate(new ColumnVisibility("(" + quote("A#C") + ")")));
+  }
+  
+  @Test
+  public void testQuote() {
+    assertEquals("\"A#C\"", quote("A#C"));
+    assertEquals("\"A\\\"C\"", quote("A\"C"));
+    assertEquals("\"A\\\"\\\\C\"", quote("A\"\\C"));
+    assertEquals("ACS", quote("ACS"));
+    assertEquals("\"九\"", quote("九"));
+    assertEquals("\"五十\"", quote("五十"));
+  }
+  
+  @Test
+  public void testNonAscii() throws VisibilityParseException, UnsupportedEncodingException {
+    VisibilityEvaluator ct = new VisibilityEvaluator(new Authorizations(Charset.forName("UTF-8"), "五", "六", "八", "九", "五十"));
+    
+    assertTrue(ct.evaluate(new ColumnVisibility(quote("五") + "|" + quote("四"), "UTF-8")));
+    assertFalse(ct.evaluate(new ColumnVisibility(quote("五") + "&" + quote("四"), "UTF-8")));
+    assertTrue(ct.evaluate(new ColumnVisibility(quote("五") + "&(" + quote("四") + "|" + quote("九") + ")", "UTF-8")));
+    assertTrue(ct.evaluate(new ColumnVisibility("\"五\"&(\"四\"|\"五十\")", "UTF-8")));
+    assertFalse(ct.evaluate(new ColumnVisibility(quote("五") + "&(" + quote("四") + "|" + quote("三") + ")", "UTF-8")));
+    assertFalse(ct.evaluate(new ColumnVisibility("\"五\"&(\"四\"|\"三\")", "UTF-8")));
+  }
 }

Modified: accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/util/shell/command/FormatterCommandTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/util/shell/command/FormatterCommandTest.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/util/shell/command/FormatterCommandTest.java (original)
+++ accumulo/branches/ACCUMULO-259/core/src/test/java/org/apache/accumulo/core/util/shell/command/FormatterCommandTest.java Sat Sep 29 05:42:59 2012
@@ -50,14 +50,14 @@ public class FormatterCommandTest {
     // Keep the Shell AUDIT log off the test output
     Logger.getLogger(Shell.class).setLevel(Level.WARN);
     
-    String[] args = new String[] {"-fake", "-u", "root", "-p", "passwd"};
+    final String[] args = new String[] {"-fake", "-u", "root", "-p", "passwd"};
    
-    String[] commands = createCommands();
+    final String[] commands = createCommands();
     
     in = MockShell.makeCommands(commands);
     writer = new StringWriter();
     
-    MockShell shell = new MockShell(in, writer);
+    final MockShell shell = new MockShell(in, writer);
     shell.config(args);
     
     // Can't call createtable in the shell with MockAccumulo
@@ -66,24 +66,23 @@ public class FormatterCommandTest {
     try {
       shell.start();
     } catch (Exception e) {
-      System.err.println(e.getMessage());
       Assert.fail("Exception while running commands: " + e.getMessage());
     } 
     
     shell.getReader().flushConsole();
     
-    String[] output = StringUtils.split(writer.toString(), '\n');
+    final String[] output = StringUtils.split(writer.toString(), '\n');
    
     boolean formatterOn = false;
     
-    String[] expectedDefault = new String[] {
+    final String[] expectedDefault = new String[] {
         "row cf:cq []    1234abcd",
         "row cf1:cq1 []    9876fedc",
         "row2 cf:cq []    13579bdf",
         "row2 cf1:cq []    2468ace"
     };
     
-    String[] expectedFormatted = new String[] {
+    final String[] expectedFormatted = new String[] {
         "row cf:cq []    0x31 0x32 0x33 0x34 0x61 0x62 0x63 0x64",
         "row cf1:cq1 []    0x39 0x38 0x37 0x36 0x66 0x65 0x64 0x63",
         "row2 cf:cq []    0x31 0x33 0x35 0x37 0x39 0x62 0x64 0x66",
@@ -92,7 +91,7 @@ public class FormatterCommandTest {
     
     int outputIndex = 0;
     while (outputIndex < output.length) {
-      String line = output[outputIndex];
+      final String line = output[outputIndex];
       
       if (line.startsWith("root@mock-instance")) {
         if (line.contains("formatter -t test -f org.apache.accumulo.core.util.shell.command.FormatterCommandTest$HexFormatter")) {
@@ -112,10 +111,8 @@ public class FormatterCommandTest {
         }
         
         // Ensure each output is what we expected
-        while (expectedIndex + outputIndex < output.length &&
-            expectedIndex < expectedFormatted.length) {
+        while (expectedIndex + outputIndex < output.length && expectedIndex < expectedFormatted.length) {
           Assert.assertEquals(comparisonData[expectedIndex].trim(), output[expectedIndex + outputIndex].trim());
-          
           expectedIndex++;
         }
         
@@ -148,8 +145,8 @@ public class FormatterCommandTest {
     private Iterator<Entry<Key, Value>> iter = null;
     private boolean printTs = false;
 
-    private final String tab = "\t";
-    private final String newline = "\n";
+    private final static String tab = "\t";
+    private final static String newline = "\n";
     
     public HexFormatter() {}
     
@@ -166,7 +163,7 @@ public class FormatterCommandTest {
      */
     @Override
     public String next() {
-      Entry<Key, Value> entry = iter.next();
+      final Entry<Key, Value> entry = iter.next();
       
       String key;
       
@@ -177,16 +174,16 @@ public class FormatterCommandTest {
         key = entry.getKey().toStringNoTime();
       }
       
-      Value v = entry.getValue();
+      final Value v = entry.getValue();
       
       // Approximate how much space we'll need
-      StringBuilder sb = new StringBuilder(key.length() + v.getSize() * 5); 
+      final StringBuilder sb = new StringBuilder(key.length() + v.getSize() * 5); 
       
       sb.append(key).append(tab);
       
       for (byte b : v.get()) {
         if ((b >= 48 && b <= 57) || (b >= 97 || b <= 102)) {
-          sb.append(String.format("0x%x ", new Integer(b)));
+          sb.append(String.format("0x%x ", Integer.valueOf(b)));
         }
       }
       
@@ -200,18 +197,16 @@ public class FormatterCommandTest {
      */
     @Override
     public void remove() {
-      return;
     }
 
     /* (non-Javadoc)
      * @see org.apache.accumulo.core.util.format.Formatter#initialize(java.lang.Iterable, boolean)
      */
     @Override
-    public void initialize(Iterable<Entry<Key,Value>> scanner, boolean printTimestamps) {
+    public void initialize(final Iterable<Entry<Key,Value>> scanner, final boolean printTimestamps) {
       this.iter = scanner.iterator();
       this.printTs = printTimestamps;
     }
-    
   }
   
 }

Propchange: accumulo/branches/ACCUMULO-259/docs/
------------------------------------------------------------------------------
--- svn:ignore (original)
+++ svn:ignore Sat Sep 29 05:42:59 2012
@@ -1,3 +1,4 @@
 config.html
+accumulo_developer_manual.pdf
 accumulo_user_manual.pdf
 apidocs

Modified: accumulo/branches/ACCUMULO-259/docs/examples/README.bloom
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/docs/examples/README.bloom?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/docs/examples/README.bloom (original)
+++ accumulo/branches/ACCUMULO-259/docs/examples/README.bloom Sat Sep 29 05:42:59 2012
@@ -186,10 +186,10 @@ table has in HDFS.  This assumes Accumul
     -rw-r--r--   3 username supergroup   52436176 2012-01-10 14:01 /accumulo/tables/o8/default_tablet/F00000dk.rf
     -rw-r--r--   3 username supergroup   52850173 2012-01-10 14:02 /accumulo/tables/o8/default_tablet/F00000dl.rf
 
-Running the PrintInfo command shows that one of the files has a bloom filter
+Running the rfile-info command shows that one of the files has a bloom filter
 and its 1.5MB.
 
-    $ ./bin/accumulo org.apache.accumulo.core.file.rfile.PrintInfo /accumulo/tables/o8/default_tablet/F00000dj.rf
+    $ ./bin/accumulo rfile-info /accumulo/tables/o8/default_tablet/F00000dj.rf
     Locality group         : <DEFAULT>
 	Start block          : 0
 	Num   blocks         : 752

Propchange: accumulo/branches/ACCUMULO-259/docs/src/user_manual/
------------------------------------------------------------------------------
--- svn:ignore (original)
+++ svn:ignore Sat Sep 29 05:42:59 2012
@@ -1,2 +1,5 @@
-accumulo_user_manual.toc
+accumulo_user_manual.aux
+accumulo_user_manual.log
 accumulo_user_manual.out
+accumulo_user_manual.pdf
+accumulo_user_manual.toc

Propchange: accumulo/branches/ACCUMULO-259/docs/src/user_manual/appendices/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Sat Sep 29 05:42:59 2012
@@ -0,0 +1 @@
+*.aux

Modified: accumulo/branches/ACCUMULO-259/docs/src/user_manual/build.sh
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/docs/src/user_manual/build.sh?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/docs/src/user_manual/build.sh (original)
+++ accumulo/branches/ACCUMULO-259/docs/src/user_manual/build.sh Sat Sep 29 05:42:59 2012
@@ -21,7 +21,9 @@ bin=`cd "$bin"; pwd`
 cd $bin
 
 if [ `which pdflatex | wc -l` -eq 0 ]; then
-  echo 'Missing pdflatex command. Please install.' 1>&2
+    echo "*********************************************" 1>&2
+    echo '* Missing pdflatex command. Please install. *' 1>&2
+    echo "*********************************************" 1>&2
   exit 0;
 fi
 
@@ -29,6 +31,9 @@ if [ -f ../../accumulo_user_manual.pdf ]
 then
   if [ `find . -name '*.tex' -newer ../../accumulo_user_manual.pdf | wc -l` -eq 0 ] 
   then
+    echo "***********************************************" 1>&2
+    echo "* User Manual is up-to-date.                  *" 1>&2
+    echo "***********************************************" 1>&2
    exit 0
   fi
 fi

Propchange: accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Sat Sep 29 05:42:59 2012
@@ -0,0 +1 @@
+*.aux

Modified: accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/analytics.tex
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/analytics.tex?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/analytics.tex (original)
+++ accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/analytics.tex Sat Sep 29 05:42:59 2012
@@ -59,7 +59,7 @@ name does not have to be passed to the C
 \begin{verbatim}
 class MyReducer extends Reducer<WritableComparable, Writable, Text, Mutation> {
 
-    public void reduce(WritableComparable key, Iterator<Text> values, Context c) {
+    public void reduce(WritableComparable key, Iterable<Text> values, Context c) {
         
         Mutation m;
         

Modified: accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/shell.tex
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/shell.tex?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/shell.tex (original)
+++ accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/shell.tex Sat Sep 29 05:42:59 2012
@@ -25,7 +25,6 @@ The shell can be started by the followin
 \begin{verbatim}
 $ACCUMULO_HOME/bin/accumulo shell -u [username]
 \end{verbatim}
-% stupid emacs is confused by $ACCUMULO
 
 \normalsize
 

Modified: accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/table_configuration.tex
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/table_configuration.tex?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/table_configuration.tex (original)
+++ accumulo/branches/ACCUMULO-259/docs/src/user_manual/chapters/table_configuration.tex Sat Sep 29 05:42:59 2012
@@ -666,4 +666,12 @@ root@a14 cic> 
 \end{verbatim}
 \normalsize
 
+\section{Exporting Tables}
 
+Accumulo supports exporting tables for the purpose of copying tables to another
+cluster.  Exporting and importing tables preserves the tables configuration,
+splits, and logical time.  Tables are exported and then copied via the hadoop
+distcp command.  To export a table, it must be offline and stay offline while
+discp runs.  The reason it needs to stay offline is to prevent files from being
+deleted.  A table can be cloned and the clone taken offline inorder to avoid
+losing access to the table.  See docs/examples/README.example for an example.

Modified: accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java (original)
+++ accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java Sat Sep 29 05:42:59 2012
@@ -114,7 +114,7 @@ public class AdminUtil<T> {
       
       zs.unreserve(tid, 0);
       
-      System.out.printf("txid: %016x  status: %-18s  op: %-15s  locked: %-15s locking: %-15s top: %s\n", tid, status, debug, hlocks, wlocks, top);
+      System.out.printf("txid: %016x  status: %-18s  op: %-15s  locked: %-15s locking: %-15s top: %s%n", tid, status, debug, hlocks, wlocks, top);
     }
     
     if (heldLocks.size() != 0 || waitingLocks.size() != 0) {
@@ -122,10 +122,10 @@ public class AdminUtil<T> {
       System.out.println("The following locks did not have an associated FATE operation");
       System.out.println();
       for (Entry<Long,List<String>> entry : heldLocks.entrySet())
-        System.out.printf("txid: %016x  locked: %s\n", entry.getKey(), entry.getValue());
+        System.out.printf("txid: %016x  locked: %s%n", entry.getKey(), entry.getValue());
       
       for (Entry<Long,List<String>> entry : waitingLocks.entrySet())
-        System.out.printf("txid: %016x  locking: %s\n", entry.getKey(), entry.getValue());
+        System.out.printf("txid: %016x  locking: %s%n", entry.getKey(), entry.getValue());
     }
   }
   

Modified: accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java (original)
+++ accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java Sat Sep 29 05:42:59 2012
@@ -59,8 +59,8 @@ public class ZooLock implements Watcher 
   
   private String asyncLock;
   
-  public ZooLock(String zookeepers, int timeInMillis, String auth, String path) {
-    this(new ZooCache(zookeepers, timeInMillis), ZooReaderWriter.getInstance(zookeepers, timeInMillis, auth), path);
+  public ZooLock(String zookeepers, int timeInMillis, String scheme, byte[] auth, String path) {
+    this(new ZooCache(zookeepers, timeInMillis), ZooReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path);
   }
   
   protected ZooLock(ZooCache zc, IZooReaderWriter zrw, String path) {

Modified: accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooQueueLock.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooQueueLock.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooQueueLock.java (original)
+++ accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooQueueLock.java Sat Sep 29 05:42:59 2012
@@ -38,8 +38,9 @@ public class ZooQueueLock implements Que
   private String path;
   private boolean ephemeral;
   
-  public ZooQueueLock(String zookeepers, int timeInMillis, String auth, String path, boolean ephemeral) throws KeeperException, InterruptedException {
-    this(ZooReaderWriter.getRetryingInstance(zookeepers, timeInMillis, auth), path, ephemeral);
+  public ZooQueueLock(String zookeepers, int timeInMillis, String scheme, byte[] auth, String path, boolean ephemeral) throws KeeperException,
+      InterruptedException {
+    this(ZooReaderWriter.getRetryingInstance(zookeepers, timeInMillis, scheme, auth), path, ephemeral);
   }
   
   protected ZooQueueLock(IZooReaderWriter zrw, String path, boolean ephemeral) {

Modified: accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java (original)
+++ accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java Sat Sep 29 05:42:59 2012
@@ -28,12 +28,12 @@ public class ZooReader implements IZooRe
   protected String keepers;
   protected int timeout;
   
-  protected ZooKeeper getSession(String keepers, int timeout, String auth) {
-    return ZooSession.getSession(keepers, timeout, auth);
+  protected ZooKeeper getSession(String keepers, int timeout, String scheme, byte[] auth) {
+    return ZooSession.getSession(keepers, timeout, scheme, auth);
   }
   
   protected ZooKeeper getZooKeeper() {
-    return getSession(keepers, timeout, null);
+    return getSession(keepers, timeout, null, null);
   }
   
   @Override

Modified: accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java (original)
+++ accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java Sat Sep 29 05:42:59 2012
@@ -43,7 +43,8 @@ public class ZooReaderWriter extends Zoo
   
   private static ZooReaderWriter instance = null;
   private static IZooReaderWriter retryingInstance = null;
-  private final String auth;
+  private final String scheme;
+  private final byte[] auth;
   
   @Override
   public ZooKeeper getZooKeeper() {
@@ -51,12 +52,13 @@ public class ZooReaderWriter extends Zoo
     if (sm != null) {
       sm.checkPermission(ZOOWRITER_PERMISSION);
     }
-    return getSession(keepers, timeout, auth);
+    return getSession(keepers, timeout, scheme, auth);
   }
   
-  public ZooReaderWriter(String string, int timeInMillis, String auth) {
+  public ZooReaderWriter(String string, int timeInMillis, String scheme, byte[] auth) {
     super(string, timeInMillis);
-    this.auth = "accumulo:" + auth;
+    this.scheme = scheme;
+    this.auth = auth;
   }
   
   @Override
@@ -143,9 +145,9 @@ public class ZooReaderWriter extends Zoo
     } while (true);
   }
   
-  public static synchronized ZooReaderWriter getInstance(String zookeepers, int timeInMillis, String auth) {
+  public static synchronized ZooReaderWriter getInstance(String zookeepers, int timeInMillis, String scheme, byte[] auth) {
     if (instance == null)
-      instance = new ZooReaderWriter(zookeepers, timeInMillis, auth);
+      instance = new ZooReaderWriter(zookeepers, timeInMillis, scheme, auth);
     return instance;
   }
   
@@ -154,10 +156,10 @@ public class ZooReaderWriter extends Zoo
    * 
    * @return an instance that retries when Zookeeper connection errors occur.
    */
-  public static synchronized IZooReaderWriter getRetryingInstance(String zookeepers, int timeInMillis, String auth) {
+  public static synchronized IZooReaderWriter getRetryingInstance(String zookeepers, int timeInMillis, String scheme, byte[] auth) {
     
     if (retryingInstance == null) {
-      final IZooReaderWriter inst = getInstance(zookeepers, timeInMillis, auth);
+      final IZooReaderWriter inst = getInstance(zookeepers, timeInMillis, scheme, auth);
       
       InvocationHandler ih = new InvocationHandler() {
         @Override

Modified: accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java (original)
+++ accumulo/branches/ACCUMULO-259/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java Sat Sep 29 05:42:59 2012
@@ -46,8 +46,8 @@ class ZooSession {
   
   private static Map<String,ZooSessionInfo> sessions = new HashMap<String,ZooSessionInfo>();
   
-  private static String sessionKey(String keepers, int timeout, String auth) {
-    return keepers + ":" + timeout + ":" + (auth == null ? "" : auth);
+  private static String sessionKey(String keepers, int timeout, String scheme, byte[] auth) {
+    return keepers + ":" + timeout + ":" + (scheme == null ? "" : scheme) + ":" + (auth == null ? "" : new String(auth));
   }
   
   private static class ZooWatcher implements Watcher {
@@ -70,7 +70,7 @@ class ZooSession {
     
   }
   
-  public static ZooKeeper connect(String host, int timeout, String auth, Watcher watcher) {
+  public static ZooKeeper connect(String host, int timeout, String scheme, byte[] auth, Watcher watcher) {
     final int TIME_BETWEEN_CONNECT_CHECKS_MS = 100;
     final int TOTAL_CONNECT_TIME_WAIT_MS = 10 * 1000;
     boolean tryAgain = true;
@@ -84,7 +84,7 @@ class ZooSession {
         for (int i = 0; i < TOTAL_CONNECT_TIME_WAIT_MS / TIME_BETWEEN_CONNECT_CHECKS_MS && tryAgain; i++) {
           if (zooKeeper.getState().equals(States.CONNECTED)) {
             if (auth != null)
-              zooKeeper.addAuthInfo("digest", auth.getBytes());
+              zooKeeper.addAuthInfo(scheme, auth);
             tryAgain = false;
           } else
             UtilWaitThread.sleep(TIME_BETWEEN_CONNECT_CHECKS_MS);
@@ -116,15 +116,15 @@ class ZooSession {
   }
   
   public static synchronized ZooKeeper getSession(String zooKeepers, int timeout) {
-    return getSession(zooKeepers, timeout, null);
+    return getSession(zooKeepers, timeout, null, null);
   }
   
-  public static synchronized ZooKeeper getSession(String zooKeepers, int timeout, String auth) {
+  public static synchronized ZooKeeper getSession(String zooKeepers, int timeout, String scheme, byte[] auth) {
     
-    String sessionKey = sessionKey(zooKeepers, timeout, auth);
+    String sessionKey = sessionKey(zooKeepers, timeout, scheme, auth);
     
     // a read-only session can use a session with authorizations, so cache a copy for it w/out auths
-    String readOnlySessionKey = sessionKey(zooKeepers, timeout, null);
+    String readOnlySessionKey = sessionKey(zooKeepers, timeout, null, null);
     
     ZooSessionInfo zsi = sessions.get(sessionKey);
     if (zsi != null && zsi.zooKeeper.getState() == States.CLOSED) {
@@ -137,7 +137,7 @@ class ZooSession {
     if (zsi == null) {
       ZooWatcher watcher = new ZooWatcher();
       log.debug("Connecting to " + zooKeepers + " with timeout " + timeout + " with auth");
-      zsi = new ZooSessionInfo(connect(zooKeepers, timeout, auth, watcher), watcher);
+      zsi = new ZooSessionInfo(connect(zooKeepers, timeout, scheme, auth, watcher), watcher);
       sessions.put(sessionKey, zsi);
       if (auth != null && !sessions.containsKey(readOnlySessionKey))
         sessions.put(readOnlySessionKey, zsi);

Modified: accumulo/branches/ACCUMULO-259/pom.xml
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/pom.xml?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/pom.xml (original)
+++ accumulo/branches/ACCUMULO-259/pom.xml Sat Sep 29 05:42:59 2012
@@ -189,27 +189,6 @@
           </environmentVariables>
         </configuration>
       </plugin>
-
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>exec-maven-plugin</artifactId>
-        <inherited>false</inherited>
-        <configuration>
-          <executable>bash</executable>
-          <arguments>
-            <argument>docs/src/user_manual/build.sh</argument>
-          </arguments>
-        </configuration>
-        <executions>
-          <execution>
-            <id>user-manual</id>
-            <phase>prepare-package</phase>
-            <goals>
-              <goal>exec</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
     </plugins>
 
 
@@ -517,7 +496,7 @@
       <dependency>
         <groupId>org.mortbay.jetty</groupId>
         <artifactId>jetty</artifactId>
-        <version>[5.1,)</version>
+        <version>[5.1,7.0)</version>
         <scope>provided</scope>
       </dependency>
 

Propchange: accumulo/branches/ACCUMULO-259/server/
------------------------------------------------------------------------------
  Merged /accumulo/branches/1.4/server:r1356924-1358205,1358207-1382577,1382613,1388120,1388629
  Merged /accumulo/branches/1.4/src/server:r1364778,1365213,1382566,1382923,1388120
  Merged /accumulo/trunk/server:r1363474-1391624

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/client/BulkImporter.java Sat Sep 29 05:42:59 2012
@@ -748,15 +748,15 @@ public class BulkImporter {
         failedTablets.addAll(ft);
       
       sb.append("BULK IMPORT ASSIGNMENT STATISTICS\n");
-      sb.append(String.format("# of map files            : %,10d\n", numUniqueMapFiles));
-      sb.append(String.format("# map files with failures : %,10d %6.2f%s\n", completeFailures.size(), completeFailures.size() * 100.0 / numUniqueMapFiles, "%"));
-      sb.append(String.format("# failed failed map files : %,10d %s\n", failedFailures.size(), failedFailures.size() > 0 ? " <-- THIS IS BAD" : ""));
-      sb.append(String.format("# of tablets              : %,10d\n", counts.size()));
-      sb.append(String.format("# tablets imported to     : %,10d %6.2f%s\n", tabletsImportedTo, tabletsImportedTo * 100.0 / counts.size(), "%"));
-      sb.append(String.format("# tablets with failures   : %,10d %6.2f%s\n", failedTablets.size(), failedTablets.size() * 100.0 / counts.size(), "%"));
-      sb.append(String.format("min map files per tablet  : %,10d\n", min));
-      sb.append(String.format("max map files per tablet  : %,10d\n", max));
-      sb.append(String.format("avg map files per tablet  : %,10.2f (std dev = %.2f)\n", totalAssignments / (double) counts.size(), stddev));
+      sb.append(String.format("# of map files            : %,10d%n", numUniqueMapFiles));
+      sb.append(String.format("# map files with failures : %,10d %6.2f%s%n", completeFailures.size(), completeFailures.size() * 100.0 / numUniqueMapFiles, "%"));
+      sb.append(String.format("# failed failed map files : %,10d %s%n", failedFailures.size(), failedFailures.size() > 0 ? " <-- THIS IS BAD" : ""));
+      sb.append(String.format("# of tablets              : %,10d%n", counts.size()));
+      sb.append(String.format("# tablets imported to     : %,10d %6.2f%s%n", tabletsImportedTo, tabletsImportedTo * 100.0 / counts.size(), "%"));
+      sb.append(String.format("# tablets with failures   : %,10d %6.2f%s%n", failedTablets.size(), failedTablets.size() * 100.0 / counts.size(), "%"));
+      sb.append(String.format("min map files per tablet  : %,10d%n", min));
+      sb.append(String.format("max map files per tablet  : %,10d%n", max));
+      sb.append(String.format("avg map files per tablet  : %,10.2f (std dev = %.2f)%n", totalAssignments / (double) counts.size(), stddev));
       return sb.toString();
     }
   }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/gc/GarbageCollectWriteAheadLogs.java Sat Sep 29 05:42:59 2012
@@ -37,7 +37,6 @@ import org.apache.accumulo.core.gc.thrif
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.security.SecurityConstants;

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java Sat Sep 29 05:42:59 2012
@@ -41,6 +41,7 @@ import org.apache.accumulo.core.Constant
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IsolatedScanner;
@@ -56,7 +57,6 @@ import org.apache.accumulo.core.data.Ran
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileUtil;
-import org.apache.accumulo.core.gc.thrift.GCMonitorService;
 import org.apache.accumulo.core.gc.thrift.GCMonitorService.Iface;
 import org.apache.accumulo.core.gc.thrift.GCMonitorService.Processor;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
@@ -64,7 +64,6 @@ import org.apache.accumulo.core.gc.thrif
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.thrift.AuthInfo;
 import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.accumulo.core.util.ServerServices;
 import org.apache.accumulo.core.util.ServerServices.Service;
@@ -256,10 +255,10 @@ public class SimpleGarbageCollector impl
         // STEP 3: delete files
         if (safemode) {
           if (verbose)
-            System.out.println("SAFEMODE: There are " + candidates.size() + " data file candidates marked for deletion.\n"
-                + "          Examine the log files to identify them.\n" + "          They can be removed by executing: bin/accumulo gc --offline\n"
-                + "WARNING:  Do not run the garbage collector in offline mode unless you are positive\n"
-                + "          that the accumulo METADATA table is in a clean state, or that accumulo\n"
+            System.out.println("SAFEMODE: There are " + candidates.size() + " data file candidates marked for deletion.%n"
+                + "          Examine the log files to identify them.%n" + "          They can be removed by executing: bin/accumulo gc --offline%n"
+                + "WARNING:  Do not run the garbage collector in offline mode unless you are positive%n"
+                + "          that the accumulo METADATA table is in a clean state, or that accumulo%n"
                 + "          has not yet been run, in the case of an upgrade.");
           log.info("SAFEMODE: Listing all data file candidates for deletion");
           for (String s : candidates)
@@ -503,7 +502,7 @@ public class SimpleGarbageCollector impl
     scanner.clearColumns();
     scanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
     scanner.fetchColumnFamily(Constants.METADATA_SCANFILE_COLUMN_FAMILY);
-    ColumnFQ.fetch(scanner, Constants.METADATA_DIRECTORY_COLUMN);
+    Constants.METADATA_DIRECTORY_COLUMN.fetch(scanner);
     
     TabletIterator tabletIterator = new TabletIterator(scanner, Constants.METADATA_KEYSPACE, false, true);
     
@@ -552,7 +551,7 @@ public class SimpleGarbageCollector impl
       Connector c;
       try {
         c = instance.getConnector(SecurityConstants.getSystemCredentials());
-        writer = c.createBatchWriter(Constants.METADATA_TABLE_NAME, 10000000, 60000l, 3);
+        writer = c.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
       } catch (Exception e) {
         log.error("Unable to create writer to remove file from the !METADATA table", e);
       }

Modified: accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java?rev=1391754&r1=1391753&r2=1391754&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java (original)
+++ accumulo/branches/ACCUMULO-259/server/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java Sat Sep 29 05:42:59 2012
@@ -279,6 +279,7 @@ public class LiveTServerSet implements W
             current.remove(server);
             info.cleanup();
           }
+          ZooReaderWriter.getInstance().delete(lockPath, -1);
         }
       }
       // log.debug("Current: " + current.keySet());