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 2014/06/24 18:09:30 UTC

[01/10] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Repository: accumulo
Updated Branches:
  refs/heads/1.5.2-SNAPSHOT 65782b505 -> c3de15bd4
  refs/heads/1.6.1-SNAPSHOT fd080f03f -> 76c910bbe
  refs/heads/master a57c6bdaf -> 9defedf28


Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Conflicts:
	test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java


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

Branch: refs/heads/master
Commit: fd080f03f663579c8de6058f1ed14acb38638618
Parents: 3583409 65782b5
Author: Josh Elser <el...@apache.org>
Authored: Mon Jun 23 19:46:09 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Jun 23 19:46:09 2014 -0400

----------------------------------------------------------------------
 .../accumulo/fate/zookeeper/ZooLockTest.java    | 54 +++++++++++++++-----
 1 file changed, 40 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd080f03/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
index e902818,717bf0a..d19320d
--- a/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
+++ b/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
@@@ -39,13 -41,30 +42,30 @@@ import org.junit.rules.TemporaryFolder
   * 
   */
  public class ZooLockTest {
 -  
 -  public static TemporaryFolder folder = new TemporaryFolder();
 -  
 +
 +  public static TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
 +
    private static MiniAccumuloCluster accumulo;
  
+   static class ConnectedWatcher implements Watcher {
+     volatile boolean connected = false;
+ 
+     @Override
+     public synchronized void process(WatchedEvent event) {
+       if (event.getState() == KeeperState.SyncConnected) { // For ZK >3.4.... || event.getState() == KeeperState.ConnectedReadOnly) {
+         connected = true;
+       } else {
+         connected = false;
+       }
+     }
+ 
+     public synchronized boolean isConnected() {
+       return connected;
+     }
+   }
+ 
    static class TestALW implements AsyncLockWatcher {
 -    
 +
      LockLossReason reason = null;
      boolean locked = false;
      Exception exception = null;
@@@ -84,129 -103,121 +104,121 @@@
        this.notifyAll();
      }
    }
 -  
 +
    @BeforeClass
    public static void setupMiniCluster() throws Exception {
 -    
 +
      folder.create();
 -    
 +
      Logger.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);
 -    
 +
      accumulo = new MiniAccumuloCluster(folder.getRoot(), "superSecret");
 -    
 +
      accumulo.start();
 -    
 +
    }
 -  
 +
    private static int pdCount = 0;
 -  
 +
    @Test(timeout = 10000)
    public void testDeleteParent() throws Exception {
-     accumulo.getZooKeepers();
- 
      String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 -    
 +
      ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 -    
 +
      Assert.assertFalse(zl.isLocked());
 -    
 +
      ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes());
 -    
 +
      // intentionally created parent after lock
      zk.mkdirs(parent);
 -    
 +
      zk.delete(parent, -1);
 -    
 +
      zk.mkdirs(parent);
 -    
 +
      TestALW lw = new TestALW();
 -    
 +
      zl.lockAsync(lw, "test1".getBytes());
 -    
 +
      lw.waitForChanges(1);
 -    
 +
      Assert.assertTrue(lw.locked);
      Assert.assertTrue(zl.isLocked());
      Assert.assertNull(lw.exception);
      Assert.assertNull(lw.reason);
 -    
 +
      zl.unlock();
    }
 -  
 +
    @Test(timeout = 10000)
    public void testNoParent() throws Exception {
-     accumulo.getZooKeepers();
- 
      String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 -    
 +
      ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 -    
 +
      Assert.assertFalse(zl.isLocked());
 -    
 +
      TestALW lw = new TestALW();
 -    
 +
      zl.lockAsync(lw, "test1".getBytes());
 -    
 +
      lw.waitForChanges(1);
 -    
 +
      Assert.assertFalse(lw.locked);
      Assert.assertFalse(zl.isLocked());
      Assert.assertNotNull(lw.exception);
      Assert.assertNull(lw.reason);
    }
 -  
 +
    @Test(timeout = 10000)
    public void testDeleteLock() throws Exception {
-     accumulo.getZooKeepers();
- 
      String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 -    
 +
      ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes());
      zk.mkdirs(parent);
 -    
 +
      ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 -    
 +
      Assert.assertFalse(zl.isLocked());
 -    
 +
      TestALW lw = new TestALW();
 -    
 +
      zl.lockAsync(lw, "test1".getBytes());
 -    
 +
      lw.waitForChanges(1);
 -    
 +
      Assert.assertTrue(lw.locked);
      Assert.assertTrue(zl.isLocked());
      Assert.assertNull(lw.exception);
      Assert.assertNull(lw.reason);
 -    
 +
      zk.delete(zl.getLockPath(), -1);
 -    
 +
      lw.waitForChanges(2);
 -    
 +
      Assert.assertEquals(LockLossReason.LOCK_DELETED, lw.reason);
      Assert.assertNull(lw.exception);
 -    
 +
    }
 -  
 +
    @Test(timeout = 10000)
    public void testDeleteWaiting() throws Exception {
-     accumulo.getZooKeepers();
- 
      String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 -    
 +
      ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes());
      zk.mkdirs(parent);
 -    
 +
      ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 -    
 +
      Assert.assertFalse(zl.isLocked());
 -    
 +
      TestALW lw = new TestALW();
 -    
 +
      zl.lockAsync(lw, "test1".getBytes());
 -    
 +
      lw.waitForChanges(1);
 -    
 +
      Assert.assertTrue(lw.locked);
      Assert.assertTrue(zl.isLocked());
      Assert.assertNull(lw.exception);
@@@ -251,35 -262,38 +263,38 @@@
      Assert.assertTrue(zl3.isLocked());
      Assert.assertNull(lw3.exception);
      Assert.assertNull(lw3.reason);
 -    
 +
      zl3.unlock();
 -    
 +
    }
 -  
 +
    @Test(timeout = 10000)
    public void testUnexpectedEvent() throws Exception {
-     accumulo.getZooKeepers();
- 
      String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
  
-     ZooKeeper zk = new ZooKeeper(accumulo.getZooKeepers(), 30000, null);
+     ConnectedWatcher watcher = new ConnectedWatcher();
+     ZooKeeper zk = new ZooKeeper(accumulo.getZooKeepers(), 30000, watcher);
      zk.addAuthInfo("digest", "secret".getBytes());
  
+     while (!watcher.isConnected()) {
+       Thread.sleep(200);
+     }
+     
      zk.create(parent, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
 -    
 +
      ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 -    
 +
      Assert.assertFalse(zl.isLocked());
 -    
 +
      // would not expect data to be set on this node, but it should not cause problems.....
      zk.setData(parent, "foo".getBytes(), -1);
 -    
 +
      TestALW lw = new TestALW();
 -    
 +
      zl.lockAsync(lw, "test1".getBytes());
 -    
 +
      lw.waitForChanges(1);
 -    
 +
      Assert.assertTrue(lw.locked);
      Assert.assertTrue(zl.isLocked());
      Assert.assertNull(lw.exception);
@@@ -300,12 -314,17 +315,17 @@@
    @Test(timeout = 10000)
    public void testTryLock() throws Exception {
      String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 -    
 +
      ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 1000, "digest", "secret".getBytes(), parent);
- 
-     ZooKeeper zk = new ZooKeeper(accumulo.getZooKeepers(), 1000, null);
+     
+     ConnectedWatcher watcher = new ConnectedWatcher();
+     ZooKeeper zk = new ZooKeeper(accumulo.getZooKeepers(), 1000, watcher);
      zk.addAuthInfo("digest", "secret".getBytes());
  
+     while (!watcher.isConnected()) {
+       Thread.sleep(200);
+     }
+     
      for (int i = 0; i < 10; i++) {
        zk.create(parent, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
        zk.delete(parent, -1);
@@@ -332,17 -351,23 +352,23 @@@
    @Test(timeout = 10000)
    public void testChangeData() throws Exception {
      String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
-     ZooKeeper zk = new ZooKeeper(accumulo.getZooKeepers(), 1000, null);
+     ConnectedWatcher watcher = new ConnectedWatcher();
+     ZooKeeper zk = new ZooKeeper(accumulo.getZooKeepers(), 1000, watcher);
      zk.addAuthInfo("digest", "secret".getBytes());
+ 
+     while (!watcher.isConnected()) {
+       Thread.sleep(200);
+     }
+     
      zk.create(parent, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
 -    
 +
      ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 1000, "digest", "secret".getBytes(), parent);
 -    
 +
      TestALW lw = new TestALW();
 -    
 +
      zl.lockAsync(lw, "test1".getBytes());
      Assert.assertEquals("test1", new String(zk.getData(zl.getLockPath(), null, null)));
 -    
 +
      zl.replaceLockData("test2".getBytes());
      Assert.assertEquals("test2", new String(zk.getData(zl.getLockPath(), null, null)));
    }


[03/10] git commit: ACCUMULO-2501 Add deepCopy to RowFilter

Posted by el...@apache.org.
ACCUMULO-2501 Add deepCopy to RowFilter

Adds a base implementation of deepCopy to org.apache.accumulo.core.iterators.user.RowFilter which sets the iterator source and decisionIterator prope This allows custom RowFilters to be chained together with minimal extra effort.

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/1.6.1-SNAPSHOT
Commit: fee209e82ffd25efb3c7371aa7472a880fa8eacf
Parents: 65782b5
Author: Russ Weeks <rw...@newbrightidea.com>
Authored: Sun Mar 30 17:38:33 2014 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jun 24 10:49:27 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/iterators/user/RowFilter.java |  15 ++-
 .../core/iterators/user/RowFilterTest.java      | 102 +++++++++++++++----
 2 files changed, 97 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fee209e8/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
index 2d2fa74..27cc3f1 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
@@ -138,7 +138,20 @@ public abstract class RowFilter extends WrappingIterator {
     super.init(source, options, env);
     this.decisionIterator = new RowIterator(source.deepCopy(env));
   }
-  
+
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+      RowFilter newInstance;
+      try {
+          newInstance = getClass().newInstance();
+      } catch (Exception e) {
+          throw new RuntimeException(e);
+      }
+      newInstance.setSource(getSource().deepCopy(env));
+      newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
+      return newInstance;
+  }
+
   @Override
   public boolean hasTop() {
     return hasTop && super.hasTop();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fee209e8/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
index 229e395..4532485 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
@@ -17,8 +17,7 @@
 package org.apache.accumulo.core.iterators.user;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashSet;
+import java.util.*;
 import java.util.Map.Entry;
 
 import junit.framework.TestCase;
@@ -81,13 +80,31 @@ public class RowFilterTest extends TestCase {
     
   }
 
-  public void test1() throws Exception {
-    MockInstance instance = new MockInstance("rft1");
-    Connector conn = instance.getConnector("", new PasswordToken(""));
-    
-    conn.tableOperations().create("table1");
-    BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
-    
+  public static class RowZeroOrOneFilter extends RowFilter {
+    private static final Set<String> passRows = new HashSet<String>(Arrays.asList("0", "1"));
+    @Override
+    public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
+      return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
+    }
+  }
+
+  public static class RowOneOrTwoFilter extends RowFilter {
+    private static final Set<String> passRows = new HashSet<String>(Arrays.asList("1", "2"));
+    @Override
+    public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
+      return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
+    }
+  }
+
+  public static class TrueFilter extends RowFilter {
+    @Override
+    public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
+      return true;
+    }
+  }
+
+  public List<Mutation> createMutations() {
+    List<Mutation> mutations = new LinkedList<Mutation>();
     Mutation m = new Mutation("0");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "1");
@@ -100,23 +117,23 @@ public class RowFilterTest extends TestCase {
     m.put("cf1", "cq9", "1");
     m.put("cf2", "cq1", "1");
     m.put("cf2", "cq2", "1");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("1");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "2");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("2");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "1");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("3");
     m.put("cf1", "cq1", "0");
     m.put("cf1", "cq2", "2");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("4");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "1");
@@ -129,8 +146,21 @@ public class RowFilterTest extends TestCase {
     m.put("cf1", "cq9", "1");
     m.put("cf2", "cq1", "1");
     m.put("cf2", "cq2", "1");
-    bw.addMutation(m);
 
+    mutations.add(m);
+    return mutations;
+  }
+
+  public void test1() throws Exception {
+    MockInstance instance = new MockInstance("rft1");
+    Connector conn = instance.getConnector("", new PasswordToken(""));
+    
+    conn.tableOperations().create("table1");
+    BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
+    
+    for (Mutation m: createMutations()) {
+      bw.addMutation(m);
+    }
     IteratorSetting is = new IteratorSetting(40, SummingRowFilter.class);
     conn.tableOperations().attachIterator("table1", is);
 
@@ -163,7 +193,41 @@ public class RowFilterTest extends TestCase {
     assertEquals(new HashSet<String>(Arrays.asList("4")), getRows(scanner));
 
   }
-  
+
+  public void testChainedRowFilters() throws Exception {
+    MockInstance instance = new MockInstance("rft1");
+    Connector conn = instance.getConnector("", new PasswordToken(""));
+
+    conn.tableOperations().create("chained_row_filters");
+    BatchWriter bw = conn.createBatchWriter("chained_row_filters", new BatchWriterConfig());
+    for (Mutation m: createMutations()) {
+      bw.addMutation(m);
+    }
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1",
+        TrueFilter.class));
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2",
+        TrueFilter.class));
+    Scanner scanner = conn.createScanner("chained_row_filters", Constants.NO_AUTHS);
+    assertEquals(new HashSet<String>(Arrays.asList("0", "1", "2", "3", "4")), getRows(scanner));
+  }
+
+  public void testFilterConjunction() throws Exception {
+    MockInstance instance = new MockInstance("rft1");
+    Connector conn = instance.getConnector("", new PasswordToken(""));
+
+    conn.tableOperations().create("filter_conjunction");
+    BatchWriter bw = conn.createBatchWriter("filter_conjunction", new BatchWriterConfig());
+    for (Mutation m: createMutations()) {
+      bw.addMutation(m);
+    }
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne",
+        RowZeroOrOneFilter.class));
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo",
+        RowOneOrTwoFilter.class));
+    Scanner scanner = conn.createScanner("filter_conjunction", Constants.NO_AUTHS);
+    assertEquals(new HashSet<String>(Arrays.asList("1")), getRows(scanner));
+  }
+
   private HashSet<String> getRows(Scanner scanner) {
     HashSet<String> rows = new HashSet<String>();
     for (Entry<Key,Value> entry : scanner) {


[02/10] git commit: ACCUMULO-2501 Add deepCopy to RowFilter

Posted by el...@apache.org.
ACCUMULO-2501 Add deepCopy to RowFilter

Adds a base implementation of deepCopy to org.apache.accumulo.core.iterators.user.RowFilter which sets the iterator source and decisionIterator prope This allows custom RowFilters to be chained together with minimal extra effort.

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/1.5.2-SNAPSHOT
Commit: fee209e82ffd25efb3c7371aa7472a880fa8eacf
Parents: 65782b5
Author: Russ Weeks <rw...@newbrightidea.com>
Authored: Sun Mar 30 17:38:33 2014 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jun 24 10:49:27 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/iterators/user/RowFilter.java |  15 ++-
 .../core/iterators/user/RowFilterTest.java      | 102 +++++++++++++++----
 2 files changed, 97 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fee209e8/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
index 2d2fa74..27cc3f1 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
@@ -138,7 +138,20 @@ public abstract class RowFilter extends WrappingIterator {
     super.init(source, options, env);
     this.decisionIterator = new RowIterator(source.deepCopy(env));
   }
-  
+
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+      RowFilter newInstance;
+      try {
+          newInstance = getClass().newInstance();
+      } catch (Exception e) {
+          throw new RuntimeException(e);
+      }
+      newInstance.setSource(getSource().deepCopy(env));
+      newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
+      return newInstance;
+  }
+
   @Override
   public boolean hasTop() {
     return hasTop && super.hasTop();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fee209e8/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
index 229e395..4532485 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
@@ -17,8 +17,7 @@
 package org.apache.accumulo.core.iterators.user;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashSet;
+import java.util.*;
 import java.util.Map.Entry;
 
 import junit.framework.TestCase;
@@ -81,13 +80,31 @@ public class RowFilterTest extends TestCase {
     
   }
 
-  public void test1() throws Exception {
-    MockInstance instance = new MockInstance("rft1");
-    Connector conn = instance.getConnector("", new PasswordToken(""));
-    
-    conn.tableOperations().create("table1");
-    BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
-    
+  public static class RowZeroOrOneFilter extends RowFilter {
+    private static final Set<String> passRows = new HashSet<String>(Arrays.asList("0", "1"));
+    @Override
+    public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
+      return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
+    }
+  }
+
+  public static class RowOneOrTwoFilter extends RowFilter {
+    private static final Set<String> passRows = new HashSet<String>(Arrays.asList("1", "2"));
+    @Override
+    public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
+      return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
+    }
+  }
+
+  public static class TrueFilter extends RowFilter {
+    @Override
+    public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
+      return true;
+    }
+  }
+
+  public List<Mutation> createMutations() {
+    List<Mutation> mutations = new LinkedList<Mutation>();
     Mutation m = new Mutation("0");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "1");
@@ -100,23 +117,23 @@ public class RowFilterTest extends TestCase {
     m.put("cf1", "cq9", "1");
     m.put("cf2", "cq1", "1");
     m.put("cf2", "cq2", "1");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("1");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "2");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("2");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "1");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("3");
     m.put("cf1", "cq1", "0");
     m.put("cf1", "cq2", "2");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("4");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "1");
@@ -129,8 +146,21 @@ public class RowFilterTest extends TestCase {
     m.put("cf1", "cq9", "1");
     m.put("cf2", "cq1", "1");
     m.put("cf2", "cq2", "1");
-    bw.addMutation(m);
 
+    mutations.add(m);
+    return mutations;
+  }
+
+  public void test1() throws Exception {
+    MockInstance instance = new MockInstance("rft1");
+    Connector conn = instance.getConnector("", new PasswordToken(""));
+    
+    conn.tableOperations().create("table1");
+    BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
+    
+    for (Mutation m: createMutations()) {
+      bw.addMutation(m);
+    }
     IteratorSetting is = new IteratorSetting(40, SummingRowFilter.class);
     conn.tableOperations().attachIterator("table1", is);
 
@@ -163,7 +193,41 @@ public class RowFilterTest extends TestCase {
     assertEquals(new HashSet<String>(Arrays.asList("4")), getRows(scanner));
 
   }
-  
+
+  public void testChainedRowFilters() throws Exception {
+    MockInstance instance = new MockInstance("rft1");
+    Connector conn = instance.getConnector("", new PasswordToken(""));
+
+    conn.tableOperations().create("chained_row_filters");
+    BatchWriter bw = conn.createBatchWriter("chained_row_filters", new BatchWriterConfig());
+    for (Mutation m: createMutations()) {
+      bw.addMutation(m);
+    }
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1",
+        TrueFilter.class));
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2",
+        TrueFilter.class));
+    Scanner scanner = conn.createScanner("chained_row_filters", Constants.NO_AUTHS);
+    assertEquals(new HashSet<String>(Arrays.asList("0", "1", "2", "3", "4")), getRows(scanner));
+  }
+
+  public void testFilterConjunction() throws Exception {
+    MockInstance instance = new MockInstance("rft1");
+    Connector conn = instance.getConnector("", new PasswordToken(""));
+
+    conn.tableOperations().create("filter_conjunction");
+    BatchWriter bw = conn.createBatchWriter("filter_conjunction", new BatchWriterConfig());
+    for (Mutation m: createMutations()) {
+      bw.addMutation(m);
+    }
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne",
+        RowZeroOrOneFilter.class));
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo",
+        RowOneOrTwoFilter.class));
+    Scanner scanner = conn.createScanner("filter_conjunction", Constants.NO_AUTHS);
+    assertEquals(new HashSet<String>(Arrays.asList("1")), getRows(scanner));
+  }
+
   private HashSet<String> getRows(Scanner scanner) {
     HashSet<String> rows = new HashSet<String>();
     for (Entry<Key,Value> entry : scanner) {


[10/10] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'


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

Branch: refs/heads/master
Commit: 9defedf28c9b22390b7105981eb9dcb939d4a222
Parents: a57c6bd 76c910b
Author: Josh Elser <el...@apache.org>
Authored: Tue Jun 24 11:57:09 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jun 24 11:57:09 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/iterators/user/RowFilter.java |  15 +-
 .../core/iterators/user/RowFilterTest.java      | 215 +++++++++++++++----
 2 files changed, 193 insertions(+), 37 deletions(-)
----------------------------------------------------------------------



[05/10] git commit: ACCUMULO-2501 Fix some formatting and add an explicit test for deepCopy on RowFilter

Posted by el...@apache.org.
ACCUMULO-2501 Fix some formatting and add an explicit test for deepCopy on RowFilter


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

Branch: refs/heads/1.5.2-SNAPSHOT
Commit: c3de15bd416bf2476afcee9a3cda8397f433531c
Parents: fee209e
Author: Josh Elser <el...@apache.org>
Authored: Tue Jun 24 11:42:59 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jun 24 11:42:59 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/iterators/user/RowFilter.java |  18 +--
 .../core/iterators/user/RowFilterTest.java      | 141 +++++++++++++++----
 2 files changed, 119 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3de15bd/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
index 27cc3f1..9c4edc2 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
@@ -141,15 +141,15 @@ public abstract class RowFilter extends WrappingIterator {
 
   @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-      RowFilter newInstance;
-      try {
-          newInstance = getClass().newInstance();
-      } catch (Exception e) {
-          throw new RuntimeException(e);
-      }
-      newInstance.setSource(getSource().deepCopy(env));
-      newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
-      return newInstance;
+    RowFilter newInstance;
+    try {
+      newInstance = getClass().newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    newInstance.setSource(getSource().deepCopy(env));
+    newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
+    return newInstance;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3de15bd/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
index 4532485..9af6340 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
@@ -16,11 +16,18 @@
  */
 package org.apache.accumulo.core.iterators.user;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
-import java.util.*;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Map.Entry;
-
-import junit.framework.TestCase;
+import java.util.Set;
+import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -31,28 +38,32 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.ColumnUpdate;
 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.DefaultIteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.hadoop.io.Text;
+import org.junit.Test;
 
 /**
  * 
  */
 
-public class RowFilterTest extends TestCase {
-  
+public class RowFilterTest {
+
   public static class SummingRowFilter extends RowFilter {
-    
+
     @Override
     public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
       int sum = 0;
       int sum2 = 0;
-      
+
       Key firstKey = null;
-      
+
       if (rowIterator.hasTop()) {
         firstKey = new Key(rowIterator.getTopKey());
       }
@@ -61,27 +72,28 @@ public class RowFilterTest extends TestCase {
         sum += Integer.parseInt(rowIterator.getTopValue().toString());
         rowIterator.next();
       }
-      
+
       // ensure that seeks are confined to the row
       rowIterator.seek(new Range(), new HashSet<ByteSequence>(), false);
       while (rowIterator.hasTop()) {
         sum2 += Integer.parseInt(rowIterator.getTopValue().toString());
         rowIterator.next();
       }
-      
+
       rowIterator.seek(new Range(firstKey.getRow(), false, null, true), new HashSet<ByteSequence>(), false);
       while (rowIterator.hasTop()) {
         sum2 += Integer.parseInt(rowIterator.getTopValue().toString());
         rowIterator.next();
       }
-      
+
       return sum == 2 && sum2 == 2;
     }
-    
+
   }
 
   public static class RowZeroOrOneFilter extends RowFilter {
     private static final Set<String> passRows = new HashSet<String>(Arrays.asList("0", "1"));
+
     @Override
     public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
       return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
@@ -90,6 +102,7 @@ public class RowFilterTest extends TestCase {
 
   public static class RowOneOrTwoFilter extends RowFilter {
     private static final Set<String> passRows = new HashSet<String>(Arrays.asList("1", "2"));
+
     @Override
     public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
       return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
@@ -151,14 +164,36 @@ public class RowFilterTest extends TestCase {
     return mutations;
   }
 
+  public TreeMap<Key,Value> createKeyValues() {
+    List<Mutation> mutations = createMutations();
+    TreeMap<Key,Value> keyValues = new TreeMap<Key,Value>();
+
+    final Text cf = new Text(), cq = new Text();
+    for (Mutation m : mutations) {
+      final Text row = new Text(m.getRow());
+      for (ColumnUpdate update : m.getUpdates()) {
+        cf.set(update.getColumnFamily());
+        cq.set(update.getColumnQualifier());
+
+        Key k = new Key(row, cf, cq);
+        Value v = new Value(update.getValue());
+
+        keyValues.put(k, v);
+      }
+    }
+
+    return keyValues;
+  }
+
+  @Test
   public void test1() throws Exception {
     MockInstance instance = new MockInstance("rft1");
     Connector conn = instance.getConnector("", new PasswordToken(""));
-    
+
     conn.tableOperations().create("table1");
     BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
-    
-    for (Mutation m: createMutations()) {
+
+    for (Mutation m : createMutations()) {
       bw.addMutation(m);
     }
     IteratorSetting is = new IteratorSetting(40, SummingRowFilter.class);
@@ -166,26 +201,26 @@ public class RowFilterTest extends TestCase {
 
     Scanner scanner = conn.createScanner("table1", Constants.NO_AUTHS);
     assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(scanner));
-    
+
     scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
     assertEquals(new HashSet<String>(Arrays.asList("1", "3")), getRows(scanner));
-    
+
     scanner.clearColumns();
     scanner.fetchColumn(new Text("cf1"), new Text("cq1"));
     assertEquals(new HashSet<String>(), getRows(scanner));
-    
+
     scanner.setRange(new Range("0", "4"));
     scanner.clearColumns();
     assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(scanner));
-    
+
     scanner.setRange(new Range("2"));
     scanner.clearColumns();
     assertEquals(new HashSet<String>(Arrays.asList("2")), getRows(scanner));
-    
+
     scanner.setRange(new Range("4"));
     scanner.clearColumns();
     assertEquals(new HashSet<String>(), getRows(scanner));
-    
+
     scanner.setRange(new Range("4"));
     scanner.clearColumns();
     scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
@@ -194,40 +229,84 @@ public class RowFilterTest extends TestCase {
 
   }
 
+  @Test
   public void testChainedRowFilters() throws Exception {
     MockInstance instance = new MockInstance("rft1");
     Connector conn = instance.getConnector("", new PasswordToken(""));
 
     conn.tableOperations().create("chained_row_filters");
     BatchWriter bw = conn.createBatchWriter("chained_row_filters", new BatchWriterConfig());
-    for (Mutation m: createMutations()) {
+    for (Mutation m : createMutations()) {
       bw.addMutation(m);
     }
-    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1",
-        TrueFilter.class));
-    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2",
-        TrueFilter.class));
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1", TrueFilter.class));
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2", TrueFilter.class));
     Scanner scanner = conn.createScanner("chained_row_filters", Constants.NO_AUTHS);
     assertEquals(new HashSet<String>(Arrays.asList("0", "1", "2", "3", "4")), getRows(scanner));
   }
 
+  @Test
   public void testFilterConjunction() throws Exception {
     MockInstance instance = new MockInstance("rft1");
     Connector conn = instance.getConnector("", new PasswordToken(""));
 
     conn.tableOperations().create("filter_conjunction");
     BatchWriter bw = conn.createBatchWriter("filter_conjunction", new BatchWriterConfig());
-    for (Mutation m: createMutations()) {
+    for (Mutation m : createMutations()) {
       bw.addMutation(m);
     }
-    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne",
-        RowZeroOrOneFilter.class));
-    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo",
-        RowOneOrTwoFilter.class));
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne", RowZeroOrOneFilter.class));
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo", RowOneOrTwoFilter.class));
     Scanner scanner = conn.createScanner("filter_conjunction", Constants.NO_AUTHS);
     assertEquals(new HashSet<String>(Arrays.asList("1")), getRows(scanner));
   }
 
+  @Test
+  public void deepCopyCopiesTheSource() throws Exception {
+    SortedMapIterator source = new SortedMapIterator(createKeyValues());
+
+    RowFilter filter = new RowZeroOrOneFilter();
+    filter.init(source, Collections.<String,String> emptyMap(), new DefaultIteratorEnvironment());
+
+    filter.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+
+    // Save off the first key and value
+    Key firstKey = filter.getTopKey();
+    Value firstValue = filter.getTopValue();
+
+    // Assert that the row is valid given our filter
+    assertEquals("0", firstKey.getRow().toString());
+
+    // Read some extra data, just making sure it's all valid
+    Key lastKeyRead = null;
+    for (int i = 0; i < 5; i++) {
+      filter.next();
+      lastKeyRead = filter.getTopKey();
+      assertEquals("0", lastKeyRead.getRow().toString());
+    }
+
+    // Make a copy of the original RowFilter
+    RowFilter copy = (RowFilter) filter.deepCopy(new DefaultIteratorEnvironment());
+
+    // Because it's a copy, we should be able to safely seek this one without affecting the original
+    copy.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+
+    assertTrue("deepCopy'ed RowFilter did not have a top key", copy.hasTop());
+
+    Key firstKeyFromCopy = copy.getTopKey();
+    Value firstValueFromCopy = copy.getTopValue();
+
+    // Verify that we got the same first k-v pair we did earlier
+    assertEquals(firstKey, firstKeyFromCopy);
+    assertEquals(firstValue, firstValueFromCopy);
+
+    filter.next();
+    Key finalKeyRead = filter.getTopKey();
+
+    // Make sure we got a Key that was greater than the last Key we read from the original RowFilter
+    assertTrue("Expected next key read to be greater than the previous after deepCopy", lastKeyRead.compareTo(finalKeyRead) < 0);
+  }
+
   private HashSet<String> getRows(Scanner scanner) {
     HashSet<String> rows = new HashSet<String>();
     for (Entry<Key,Value> entry : scanner) {


[09/10] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Posted by el...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Conflicts:
	core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java


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

Branch: refs/heads/master
Commit: 76c910bbecf373a5b75c621a0051d9acf4f62ee0
Parents: fd080f0 c3de15b
Author: Josh Elser <el...@apache.org>
Authored: Tue Jun 24 11:56:59 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jun 24 11:56:59 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/iterators/user/RowFilter.java |  15 +-
 .../core/iterators/user/RowFilterTest.java      | 215 +++++++++++++++----
 2 files changed, 193 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/76c910bb/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
index 7435514,9af6340..958ac18
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
@@@ -16,13 -16,20 +16,19 @@@
   */
  package org.apache.accumulo.core.iterators.user;
  
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertTrue;
+ 
  import java.io.IOException;
  import java.util.Arrays;
+ import java.util.Collections;
  import java.util.HashSet;
+ import java.util.LinkedList;
+ import java.util.List;
  import java.util.Map.Entry;
- 
- import junit.framework.TestCase;
+ import java.util.Set;
+ import java.util.TreeMap;
  
 -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;
@@@ -35,9 -43,11 +42,12 @@@ import org.apache.accumulo.core.data.Ke
  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.DefaultIteratorEnvironment;
  import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+ import org.apache.accumulo.core.iterators.SortedMapIterator;
 +import org.apache.accumulo.core.security.Authorizations;
  import org.apache.hadoop.io.Text;
+ import org.junit.Test;
  
  /**
   * 
@@@ -129,17 -159,52 +159,52 @@@ public class RowFilterTest 
      m.put("cf1", "cq9", "1");
      m.put("cf2", "cq1", "1");
      m.put("cf2", "cq2", "1");
-     bw.addMutation(m);
-     
+ 
+     mutations.add(m);
+     return mutations;
+   }
+ 
+   public TreeMap<Key,Value> createKeyValues() {
+     List<Mutation> mutations = createMutations();
+     TreeMap<Key,Value> keyValues = new TreeMap<Key,Value>();
+ 
+     final Text cf = new Text(), cq = new Text();
+     for (Mutation m : mutations) {
+       final Text row = new Text(m.getRow());
+       for (ColumnUpdate update : m.getUpdates()) {
+         cf.set(update.getColumnFamily());
+         cq.set(update.getColumnQualifier());
+ 
+         Key k = new Key(row, cf, cq);
+         Value v = new Value(update.getValue());
+ 
+         keyValues.put(k, v);
+       }
+     }
+ 
+     return keyValues;
+   }
+ 
+   @Test
+   public void test1() throws Exception {
+     MockInstance instance = new MockInstance("rft1");
+     Connector conn = instance.getConnector("", new PasswordToken(""));
+ 
+     conn.tableOperations().create("table1");
+     BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
+ 
+     for (Mutation m : createMutations()) {
+       bw.addMutation(m);
+     }
      IteratorSetting is = new IteratorSetting(40, SummingRowFilter.class);
      conn.tableOperations().attachIterator("table1", is);
 -
 -    Scanner scanner = conn.createScanner("table1", Constants.NO_AUTHS);
 +    
 +    Scanner scanner = conn.createScanner("table1", Authorizations.EMPTY);
      assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(scanner));
-     
+ 
      scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
      assertEquals(new HashSet<String>(Arrays.asList("1", "3")), getRows(scanner));
-     
+ 
      scanner.clearColumns();
      scanner.fetchColumn(new Text("cf1"), new Text("cq1"));
      assertEquals(new HashSet<String>(), getRows(scanner));
@@@ -161,9 -226,87 +226,87 @@@
      scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
      scanner.fetchColumn(new Text("cf1"), new Text("cq4"));
      assertEquals(new HashSet<String>(Arrays.asList("4")), getRows(scanner));
 -
 +    
    }
-   
+ 
+   @Test
+   public void testChainedRowFilters() throws Exception {
+     MockInstance instance = new MockInstance("rft1");
+     Connector conn = instance.getConnector("", new PasswordToken(""));
+ 
+     conn.tableOperations().create("chained_row_filters");
+     BatchWriter bw = conn.createBatchWriter("chained_row_filters", new BatchWriterConfig());
+     for (Mutation m : createMutations()) {
+       bw.addMutation(m);
+     }
+     conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1", TrueFilter.class));
+     conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2", TrueFilter.class));
 -    Scanner scanner = conn.createScanner("chained_row_filters", Constants.NO_AUTHS);
++    Scanner scanner = conn.createScanner("chained_row_filters", Authorizations.EMPTY);
+     assertEquals(new HashSet<String>(Arrays.asList("0", "1", "2", "3", "4")), getRows(scanner));
+   }
+ 
+   @Test
+   public void testFilterConjunction() throws Exception {
+     MockInstance instance = new MockInstance("rft1");
+     Connector conn = instance.getConnector("", new PasswordToken(""));
+ 
+     conn.tableOperations().create("filter_conjunction");
+     BatchWriter bw = conn.createBatchWriter("filter_conjunction", new BatchWriterConfig());
+     for (Mutation m : createMutations()) {
+       bw.addMutation(m);
+     }
+     conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne", RowZeroOrOneFilter.class));
+     conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo", RowOneOrTwoFilter.class));
 -    Scanner scanner = conn.createScanner("filter_conjunction", Constants.NO_AUTHS);
++    Scanner scanner = conn.createScanner("filter_conjunction", Authorizations.EMPTY);
+     assertEquals(new HashSet<String>(Arrays.asList("1")), getRows(scanner));
+   }
+ 
+   @Test
+   public void deepCopyCopiesTheSource() throws Exception {
+     SortedMapIterator source = new SortedMapIterator(createKeyValues());
+ 
+     RowFilter filter = new RowZeroOrOneFilter();
+     filter.init(source, Collections.<String,String> emptyMap(), new DefaultIteratorEnvironment());
+ 
+     filter.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+ 
+     // Save off the first key and value
+     Key firstKey = filter.getTopKey();
+     Value firstValue = filter.getTopValue();
+ 
+     // Assert that the row is valid given our filter
+     assertEquals("0", firstKey.getRow().toString());
+ 
+     // Read some extra data, just making sure it's all valid
+     Key lastKeyRead = null;
+     for (int i = 0; i < 5; i++) {
+       filter.next();
+       lastKeyRead = filter.getTopKey();
+       assertEquals("0", lastKeyRead.getRow().toString());
+     }
+ 
+     // Make a copy of the original RowFilter
+     RowFilter copy = (RowFilter) filter.deepCopy(new DefaultIteratorEnvironment());
+ 
+     // Because it's a copy, we should be able to safely seek this one without affecting the original
+     copy.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+ 
+     assertTrue("deepCopy'ed RowFilter did not have a top key", copy.hasTop());
+ 
+     Key firstKeyFromCopy = copy.getTopKey();
+     Value firstValueFromCopy = copy.getTopValue();
+ 
+     // Verify that we got the same first k-v pair we did earlier
+     assertEquals(firstKey, firstKeyFromCopy);
+     assertEquals(firstValue, firstValueFromCopy);
+ 
+     filter.next();
+     Key finalKeyRead = filter.getTopKey();
+ 
+     // Make sure we got a Key that was greater than the last Key we read from the original RowFilter
+     assertTrue("Expected next key read to be greater than the previous after deepCopy", lastKeyRead.compareTo(finalKeyRead) < 0);
+   }
+ 
    private HashSet<String> getRows(Scanner scanner) {
      HashSet<String> rows = new HashSet<String>();
      for (Entry<Key,Value> entry : scanner) {


[04/10] git commit: ACCUMULO-2501 Add deepCopy to RowFilter

Posted by el...@apache.org.
ACCUMULO-2501 Add deepCopy to RowFilter

Adds a base implementation of deepCopy to org.apache.accumulo.core.iterators.user.RowFilter which sets the iterator source and decisionIterator prope This allows custom RowFilters to be chained together with minimal extra effort.

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/master
Commit: fee209e82ffd25efb3c7371aa7472a880fa8eacf
Parents: 65782b5
Author: Russ Weeks <rw...@newbrightidea.com>
Authored: Sun Mar 30 17:38:33 2014 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jun 24 10:49:27 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/iterators/user/RowFilter.java |  15 ++-
 .../core/iterators/user/RowFilterTest.java      | 102 +++++++++++++++----
 2 files changed, 97 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fee209e8/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
index 2d2fa74..27cc3f1 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
@@ -138,7 +138,20 @@ public abstract class RowFilter extends WrappingIterator {
     super.init(source, options, env);
     this.decisionIterator = new RowIterator(source.deepCopy(env));
   }
-  
+
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+      RowFilter newInstance;
+      try {
+          newInstance = getClass().newInstance();
+      } catch (Exception e) {
+          throw new RuntimeException(e);
+      }
+      newInstance.setSource(getSource().deepCopy(env));
+      newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
+      return newInstance;
+  }
+
   @Override
   public boolean hasTop() {
     return hasTop && super.hasTop();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fee209e8/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
index 229e395..4532485 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
@@ -17,8 +17,7 @@
 package org.apache.accumulo.core.iterators.user;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashSet;
+import java.util.*;
 import java.util.Map.Entry;
 
 import junit.framework.TestCase;
@@ -81,13 +80,31 @@ public class RowFilterTest extends TestCase {
     
   }
 
-  public void test1() throws Exception {
-    MockInstance instance = new MockInstance("rft1");
-    Connector conn = instance.getConnector("", new PasswordToken(""));
-    
-    conn.tableOperations().create("table1");
-    BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
-    
+  public static class RowZeroOrOneFilter extends RowFilter {
+    private static final Set<String> passRows = new HashSet<String>(Arrays.asList("0", "1"));
+    @Override
+    public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
+      return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
+    }
+  }
+
+  public static class RowOneOrTwoFilter extends RowFilter {
+    private static final Set<String> passRows = new HashSet<String>(Arrays.asList("1", "2"));
+    @Override
+    public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
+      return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
+    }
+  }
+
+  public static class TrueFilter extends RowFilter {
+    @Override
+    public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
+      return true;
+    }
+  }
+
+  public List<Mutation> createMutations() {
+    List<Mutation> mutations = new LinkedList<Mutation>();
     Mutation m = new Mutation("0");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "1");
@@ -100,23 +117,23 @@ public class RowFilterTest extends TestCase {
     m.put("cf1", "cq9", "1");
     m.put("cf2", "cq1", "1");
     m.put("cf2", "cq2", "1");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("1");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "2");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("2");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "1");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("3");
     m.put("cf1", "cq1", "0");
     m.put("cf1", "cq2", "2");
-    bw.addMutation(m);
-    
+    mutations.add(m);
+
     m = new Mutation("4");
     m.put("cf1", "cq1", "1");
     m.put("cf1", "cq2", "1");
@@ -129,8 +146,21 @@ public class RowFilterTest extends TestCase {
     m.put("cf1", "cq9", "1");
     m.put("cf2", "cq1", "1");
     m.put("cf2", "cq2", "1");
-    bw.addMutation(m);
 
+    mutations.add(m);
+    return mutations;
+  }
+
+  public void test1() throws Exception {
+    MockInstance instance = new MockInstance("rft1");
+    Connector conn = instance.getConnector("", new PasswordToken(""));
+    
+    conn.tableOperations().create("table1");
+    BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
+    
+    for (Mutation m: createMutations()) {
+      bw.addMutation(m);
+    }
     IteratorSetting is = new IteratorSetting(40, SummingRowFilter.class);
     conn.tableOperations().attachIterator("table1", is);
 
@@ -163,7 +193,41 @@ public class RowFilterTest extends TestCase {
     assertEquals(new HashSet<String>(Arrays.asList("4")), getRows(scanner));
 
   }
-  
+
+  public void testChainedRowFilters() throws Exception {
+    MockInstance instance = new MockInstance("rft1");
+    Connector conn = instance.getConnector("", new PasswordToken(""));
+
+    conn.tableOperations().create("chained_row_filters");
+    BatchWriter bw = conn.createBatchWriter("chained_row_filters", new BatchWriterConfig());
+    for (Mutation m: createMutations()) {
+      bw.addMutation(m);
+    }
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1",
+        TrueFilter.class));
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2",
+        TrueFilter.class));
+    Scanner scanner = conn.createScanner("chained_row_filters", Constants.NO_AUTHS);
+    assertEquals(new HashSet<String>(Arrays.asList("0", "1", "2", "3", "4")), getRows(scanner));
+  }
+
+  public void testFilterConjunction() throws Exception {
+    MockInstance instance = new MockInstance("rft1");
+    Connector conn = instance.getConnector("", new PasswordToken(""));
+
+    conn.tableOperations().create("filter_conjunction");
+    BatchWriter bw = conn.createBatchWriter("filter_conjunction", new BatchWriterConfig());
+    for (Mutation m: createMutations()) {
+      bw.addMutation(m);
+    }
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne",
+        RowZeroOrOneFilter.class));
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo",
+        RowOneOrTwoFilter.class));
+    Scanner scanner = conn.createScanner("filter_conjunction", Constants.NO_AUTHS);
+    assertEquals(new HashSet<String>(Arrays.asList("1")), getRows(scanner));
+  }
+
   private HashSet<String> getRows(Scanner scanner) {
     HashSet<String> rows = new HashSet<String>();
     for (Entry<Key,Value> entry : scanner) {


[07/10] git commit: ACCUMULO-2501 Fix some formatting and add an explicit test for deepCopy on RowFilter

Posted by el...@apache.org.
ACCUMULO-2501 Fix some formatting and add an explicit test for deepCopy on RowFilter


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

Branch: refs/heads/master
Commit: c3de15bd416bf2476afcee9a3cda8397f433531c
Parents: fee209e
Author: Josh Elser <el...@apache.org>
Authored: Tue Jun 24 11:42:59 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jun 24 11:42:59 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/iterators/user/RowFilter.java |  18 +--
 .../core/iterators/user/RowFilterTest.java      | 141 +++++++++++++++----
 2 files changed, 119 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3de15bd/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
index 27cc3f1..9c4edc2 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
@@ -141,15 +141,15 @@ public abstract class RowFilter extends WrappingIterator {
 
   @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-      RowFilter newInstance;
-      try {
-          newInstance = getClass().newInstance();
-      } catch (Exception e) {
-          throw new RuntimeException(e);
-      }
-      newInstance.setSource(getSource().deepCopy(env));
-      newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
-      return newInstance;
+    RowFilter newInstance;
+    try {
+      newInstance = getClass().newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    newInstance.setSource(getSource().deepCopy(env));
+    newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
+    return newInstance;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3de15bd/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
index 4532485..9af6340 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
@@ -16,11 +16,18 @@
  */
 package org.apache.accumulo.core.iterators.user;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
-import java.util.*;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Map.Entry;
-
-import junit.framework.TestCase;
+import java.util.Set;
+import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -31,28 +38,32 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.ColumnUpdate;
 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.DefaultIteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.hadoop.io.Text;
+import org.junit.Test;
 
 /**
  * 
  */
 
-public class RowFilterTest extends TestCase {
-  
+public class RowFilterTest {
+
   public static class SummingRowFilter extends RowFilter {
-    
+
     @Override
     public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
       int sum = 0;
       int sum2 = 0;
-      
+
       Key firstKey = null;
-      
+
       if (rowIterator.hasTop()) {
         firstKey = new Key(rowIterator.getTopKey());
       }
@@ -61,27 +72,28 @@ public class RowFilterTest extends TestCase {
         sum += Integer.parseInt(rowIterator.getTopValue().toString());
         rowIterator.next();
       }
-      
+
       // ensure that seeks are confined to the row
       rowIterator.seek(new Range(), new HashSet<ByteSequence>(), false);
       while (rowIterator.hasTop()) {
         sum2 += Integer.parseInt(rowIterator.getTopValue().toString());
         rowIterator.next();
       }
-      
+
       rowIterator.seek(new Range(firstKey.getRow(), false, null, true), new HashSet<ByteSequence>(), false);
       while (rowIterator.hasTop()) {
         sum2 += Integer.parseInt(rowIterator.getTopValue().toString());
         rowIterator.next();
       }
-      
+
       return sum == 2 && sum2 == 2;
     }
-    
+
   }
 
   public static class RowZeroOrOneFilter extends RowFilter {
     private static final Set<String> passRows = new HashSet<String>(Arrays.asList("0", "1"));
+
     @Override
     public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
       return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
@@ -90,6 +102,7 @@ public class RowFilterTest extends TestCase {
 
   public static class RowOneOrTwoFilter extends RowFilter {
     private static final Set<String> passRows = new HashSet<String>(Arrays.asList("1", "2"));
+
     @Override
     public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
       return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
@@ -151,14 +164,36 @@ public class RowFilterTest extends TestCase {
     return mutations;
   }
 
+  public TreeMap<Key,Value> createKeyValues() {
+    List<Mutation> mutations = createMutations();
+    TreeMap<Key,Value> keyValues = new TreeMap<Key,Value>();
+
+    final Text cf = new Text(), cq = new Text();
+    for (Mutation m : mutations) {
+      final Text row = new Text(m.getRow());
+      for (ColumnUpdate update : m.getUpdates()) {
+        cf.set(update.getColumnFamily());
+        cq.set(update.getColumnQualifier());
+
+        Key k = new Key(row, cf, cq);
+        Value v = new Value(update.getValue());
+
+        keyValues.put(k, v);
+      }
+    }
+
+    return keyValues;
+  }
+
+  @Test
   public void test1() throws Exception {
     MockInstance instance = new MockInstance("rft1");
     Connector conn = instance.getConnector("", new PasswordToken(""));
-    
+
     conn.tableOperations().create("table1");
     BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
-    
-    for (Mutation m: createMutations()) {
+
+    for (Mutation m : createMutations()) {
       bw.addMutation(m);
     }
     IteratorSetting is = new IteratorSetting(40, SummingRowFilter.class);
@@ -166,26 +201,26 @@ public class RowFilterTest extends TestCase {
 
     Scanner scanner = conn.createScanner("table1", Constants.NO_AUTHS);
     assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(scanner));
-    
+
     scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
     assertEquals(new HashSet<String>(Arrays.asList("1", "3")), getRows(scanner));
-    
+
     scanner.clearColumns();
     scanner.fetchColumn(new Text("cf1"), new Text("cq1"));
     assertEquals(new HashSet<String>(), getRows(scanner));
-    
+
     scanner.setRange(new Range("0", "4"));
     scanner.clearColumns();
     assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(scanner));
-    
+
     scanner.setRange(new Range("2"));
     scanner.clearColumns();
     assertEquals(new HashSet<String>(Arrays.asList("2")), getRows(scanner));
-    
+
     scanner.setRange(new Range("4"));
     scanner.clearColumns();
     assertEquals(new HashSet<String>(), getRows(scanner));
-    
+
     scanner.setRange(new Range("4"));
     scanner.clearColumns();
     scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
@@ -194,40 +229,84 @@ public class RowFilterTest extends TestCase {
 
   }
 
+  @Test
   public void testChainedRowFilters() throws Exception {
     MockInstance instance = new MockInstance("rft1");
     Connector conn = instance.getConnector("", new PasswordToken(""));
 
     conn.tableOperations().create("chained_row_filters");
     BatchWriter bw = conn.createBatchWriter("chained_row_filters", new BatchWriterConfig());
-    for (Mutation m: createMutations()) {
+    for (Mutation m : createMutations()) {
       bw.addMutation(m);
     }
-    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1",
-        TrueFilter.class));
-    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2",
-        TrueFilter.class));
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1", TrueFilter.class));
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2", TrueFilter.class));
     Scanner scanner = conn.createScanner("chained_row_filters", Constants.NO_AUTHS);
     assertEquals(new HashSet<String>(Arrays.asList("0", "1", "2", "3", "4")), getRows(scanner));
   }
 
+  @Test
   public void testFilterConjunction() throws Exception {
     MockInstance instance = new MockInstance("rft1");
     Connector conn = instance.getConnector("", new PasswordToken(""));
 
     conn.tableOperations().create("filter_conjunction");
     BatchWriter bw = conn.createBatchWriter("filter_conjunction", new BatchWriterConfig());
-    for (Mutation m: createMutations()) {
+    for (Mutation m : createMutations()) {
       bw.addMutation(m);
     }
-    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne",
-        RowZeroOrOneFilter.class));
-    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo",
-        RowOneOrTwoFilter.class));
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne", RowZeroOrOneFilter.class));
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo", RowOneOrTwoFilter.class));
     Scanner scanner = conn.createScanner("filter_conjunction", Constants.NO_AUTHS);
     assertEquals(new HashSet<String>(Arrays.asList("1")), getRows(scanner));
   }
 
+  @Test
+  public void deepCopyCopiesTheSource() throws Exception {
+    SortedMapIterator source = new SortedMapIterator(createKeyValues());
+
+    RowFilter filter = new RowZeroOrOneFilter();
+    filter.init(source, Collections.<String,String> emptyMap(), new DefaultIteratorEnvironment());
+
+    filter.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+
+    // Save off the first key and value
+    Key firstKey = filter.getTopKey();
+    Value firstValue = filter.getTopValue();
+
+    // Assert that the row is valid given our filter
+    assertEquals("0", firstKey.getRow().toString());
+
+    // Read some extra data, just making sure it's all valid
+    Key lastKeyRead = null;
+    for (int i = 0; i < 5; i++) {
+      filter.next();
+      lastKeyRead = filter.getTopKey();
+      assertEquals("0", lastKeyRead.getRow().toString());
+    }
+
+    // Make a copy of the original RowFilter
+    RowFilter copy = (RowFilter) filter.deepCopy(new DefaultIteratorEnvironment());
+
+    // Because it's a copy, we should be able to safely seek this one without affecting the original
+    copy.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+
+    assertTrue("deepCopy'ed RowFilter did not have a top key", copy.hasTop());
+
+    Key firstKeyFromCopy = copy.getTopKey();
+    Value firstValueFromCopy = copy.getTopValue();
+
+    // Verify that we got the same first k-v pair we did earlier
+    assertEquals(firstKey, firstKeyFromCopy);
+    assertEquals(firstValue, firstValueFromCopy);
+
+    filter.next();
+    Key finalKeyRead = filter.getTopKey();
+
+    // Make sure we got a Key that was greater than the last Key we read from the original RowFilter
+    assertTrue("Expected next key read to be greater than the previous after deepCopy", lastKeyRead.compareTo(finalKeyRead) < 0);
+  }
+
   private HashSet<String> getRows(Scanner scanner) {
     HashSet<String> rows = new HashSet<String>();
     for (Entry<Key,Value> entry : scanner) {


[08/10] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Posted by el...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Conflicts:
	core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java


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

Branch: refs/heads/1.6.1-SNAPSHOT
Commit: 76c910bbecf373a5b75c621a0051d9acf4f62ee0
Parents: fd080f0 c3de15b
Author: Josh Elser <el...@apache.org>
Authored: Tue Jun 24 11:56:59 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jun 24 11:56:59 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/iterators/user/RowFilter.java |  15 +-
 .../core/iterators/user/RowFilterTest.java      | 215 +++++++++++++++----
 2 files changed, 193 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/76c910bb/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
index 7435514,9af6340..958ac18
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
@@@ -16,13 -16,20 +16,19 @@@
   */
  package org.apache.accumulo.core.iterators.user;
  
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertTrue;
+ 
  import java.io.IOException;
  import java.util.Arrays;
+ import java.util.Collections;
  import java.util.HashSet;
+ import java.util.LinkedList;
+ import java.util.List;
  import java.util.Map.Entry;
- 
- import junit.framework.TestCase;
+ import java.util.Set;
+ import java.util.TreeMap;
  
 -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;
@@@ -35,9 -43,11 +42,12 @@@ import org.apache.accumulo.core.data.Ke
  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.DefaultIteratorEnvironment;
  import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+ import org.apache.accumulo.core.iterators.SortedMapIterator;
 +import org.apache.accumulo.core.security.Authorizations;
  import org.apache.hadoop.io.Text;
+ import org.junit.Test;
  
  /**
   * 
@@@ -129,17 -159,52 +159,52 @@@ public class RowFilterTest 
      m.put("cf1", "cq9", "1");
      m.put("cf2", "cq1", "1");
      m.put("cf2", "cq2", "1");
-     bw.addMutation(m);
-     
+ 
+     mutations.add(m);
+     return mutations;
+   }
+ 
+   public TreeMap<Key,Value> createKeyValues() {
+     List<Mutation> mutations = createMutations();
+     TreeMap<Key,Value> keyValues = new TreeMap<Key,Value>();
+ 
+     final Text cf = new Text(), cq = new Text();
+     for (Mutation m : mutations) {
+       final Text row = new Text(m.getRow());
+       for (ColumnUpdate update : m.getUpdates()) {
+         cf.set(update.getColumnFamily());
+         cq.set(update.getColumnQualifier());
+ 
+         Key k = new Key(row, cf, cq);
+         Value v = new Value(update.getValue());
+ 
+         keyValues.put(k, v);
+       }
+     }
+ 
+     return keyValues;
+   }
+ 
+   @Test
+   public void test1() throws Exception {
+     MockInstance instance = new MockInstance("rft1");
+     Connector conn = instance.getConnector("", new PasswordToken(""));
+ 
+     conn.tableOperations().create("table1");
+     BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
+ 
+     for (Mutation m : createMutations()) {
+       bw.addMutation(m);
+     }
      IteratorSetting is = new IteratorSetting(40, SummingRowFilter.class);
      conn.tableOperations().attachIterator("table1", is);
 -
 -    Scanner scanner = conn.createScanner("table1", Constants.NO_AUTHS);
 +    
 +    Scanner scanner = conn.createScanner("table1", Authorizations.EMPTY);
      assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(scanner));
-     
+ 
      scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
      assertEquals(new HashSet<String>(Arrays.asList("1", "3")), getRows(scanner));
-     
+ 
      scanner.clearColumns();
      scanner.fetchColumn(new Text("cf1"), new Text("cq1"));
      assertEquals(new HashSet<String>(), getRows(scanner));
@@@ -161,9 -226,87 +226,87 @@@
      scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
      scanner.fetchColumn(new Text("cf1"), new Text("cq4"));
      assertEquals(new HashSet<String>(Arrays.asList("4")), getRows(scanner));
 -
 +    
    }
-   
+ 
+   @Test
+   public void testChainedRowFilters() throws Exception {
+     MockInstance instance = new MockInstance("rft1");
+     Connector conn = instance.getConnector("", new PasswordToken(""));
+ 
+     conn.tableOperations().create("chained_row_filters");
+     BatchWriter bw = conn.createBatchWriter("chained_row_filters", new BatchWriterConfig());
+     for (Mutation m : createMutations()) {
+       bw.addMutation(m);
+     }
+     conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1", TrueFilter.class));
+     conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2", TrueFilter.class));
 -    Scanner scanner = conn.createScanner("chained_row_filters", Constants.NO_AUTHS);
++    Scanner scanner = conn.createScanner("chained_row_filters", Authorizations.EMPTY);
+     assertEquals(new HashSet<String>(Arrays.asList("0", "1", "2", "3", "4")), getRows(scanner));
+   }
+ 
+   @Test
+   public void testFilterConjunction() throws Exception {
+     MockInstance instance = new MockInstance("rft1");
+     Connector conn = instance.getConnector("", new PasswordToken(""));
+ 
+     conn.tableOperations().create("filter_conjunction");
+     BatchWriter bw = conn.createBatchWriter("filter_conjunction", new BatchWriterConfig());
+     for (Mutation m : createMutations()) {
+       bw.addMutation(m);
+     }
+     conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne", RowZeroOrOneFilter.class));
+     conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo", RowOneOrTwoFilter.class));
 -    Scanner scanner = conn.createScanner("filter_conjunction", Constants.NO_AUTHS);
++    Scanner scanner = conn.createScanner("filter_conjunction", Authorizations.EMPTY);
+     assertEquals(new HashSet<String>(Arrays.asList("1")), getRows(scanner));
+   }
+ 
+   @Test
+   public void deepCopyCopiesTheSource() throws Exception {
+     SortedMapIterator source = new SortedMapIterator(createKeyValues());
+ 
+     RowFilter filter = new RowZeroOrOneFilter();
+     filter.init(source, Collections.<String,String> emptyMap(), new DefaultIteratorEnvironment());
+ 
+     filter.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+ 
+     // Save off the first key and value
+     Key firstKey = filter.getTopKey();
+     Value firstValue = filter.getTopValue();
+ 
+     // Assert that the row is valid given our filter
+     assertEquals("0", firstKey.getRow().toString());
+ 
+     // Read some extra data, just making sure it's all valid
+     Key lastKeyRead = null;
+     for (int i = 0; i < 5; i++) {
+       filter.next();
+       lastKeyRead = filter.getTopKey();
+       assertEquals("0", lastKeyRead.getRow().toString());
+     }
+ 
+     // Make a copy of the original RowFilter
+     RowFilter copy = (RowFilter) filter.deepCopy(new DefaultIteratorEnvironment());
+ 
+     // Because it's a copy, we should be able to safely seek this one without affecting the original
+     copy.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+ 
+     assertTrue("deepCopy'ed RowFilter did not have a top key", copy.hasTop());
+ 
+     Key firstKeyFromCopy = copy.getTopKey();
+     Value firstValueFromCopy = copy.getTopValue();
+ 
+     // Verify that we got the same first k-v pair we did earlier
+     assertEquals(firstKey, firstKeyFromCopy);
+     assertEquals(firstValue, firstValueFromCopy);
+ 
+     filter.next();
+     Key finalKeyRead = filter.getTopKey();
+ 
+     // Make sure we got a Key that was greater than the last Key we read from the original RowFilter
+     assertTrue("Expected next key read to be greater than the previous after deepCopy", lastKeyRead.compareTo(finalKeyRead) < 0);
+   }
+ 
    private HashSet<String> getRows(Scanner scanner) {
      HashSet<String> rows = new HashSet<String>();
      for (Entry<Key,Value> entry : scanner) {


[06/10] git commit: ACCUMULO-2501 Fix some formatting and add an explicit test for deepCopy on RowFilter

Posted by el...@apache.org.
ACCUMULO-2501 Fix some formatting and add an explicit test for deepCopy on RowFilter


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

Branch: refs/heads/1.6.1-SNAPSHOT
Commit: c3de15bd416bf2476afcee9a3cda8397f433531c
Parents: fee209e
Author: Josh Elser <el...@apache.org>
Authored: Tue Jun 24 11:42:59 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jun 24 11:42:59 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/iterators/user/RowFilter.java |  18 +--
 .../core/iterators/user/RowFilterTest.java      | 141 +++++++++++++++----
 2 files changed, 119 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3de15bd/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
index 27cc3f1..9c4edc2 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java
@@ -141,15 +141,15 @@ public abstract class RowFilter extends WrappingIterator {
 
   @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-      RowFilter newInstance;
-      try {
-          newInstance = getClass().newInstance();
-      } catch (Exception e) {
-          throw new RuntimeException(e);
-      }
-      newInstance.setSource(getSource().deepCopy(env));
-      newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
-      return newInstance;
+    RowFilter newInstance;
+    try {
+      newInstance = getClass().newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    newInstance.setSource(getSource().deepCopy(env));
+    newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
+    return newInstance;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3de15bd/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
index 4532485..9af6340 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
@@ -16,11 +16,18 @@
  */
 package org.apache.accumulo.core.iterators.user;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
-import java.util.*;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Map.Entry;
-
-import junit.framework.TestCase;
+import java.util.Set;
+import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -31,28 +38,32 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.ColumnUpdate;
 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.DefaultIteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.hadoop.io.Text;
+import org.junit.Test;
 
 /**
  * 
  */
 
-public class RowFilterTest extends TestCase {
-  
+public class RowFilterTest {
+
   public static class SummingRowFilter extends RowFilter {
-    
+
     @Override
     public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
       int sum = 0;
       int sum2 = 0;
-      
+
       Key firstKey = null;
-      
+
       if (rowIterator.hasTop()) {
         firstKey = new Key(rowIterator.getTopKey());
       }
@@ -61,27 +72,28 @@ public class RowFilterTest extends TestCase {
         sum += Integer.parseInt(rowIterator.getTopValue().toString());
         rowIterator.next();
       }
-      
+
       // ensure that seeks are confined to the row
       rowIterator.seek(new Range(), new HashSet<ByteSequence>(), false);
       while (rowIterator.hasTop()) {
         sum2 += Integer.parseInt(rowIterator.getTopValue().toString());
         rowIterator.next();
       }
-      
+
       rowIterator.seek(new Range(firstKey.getRow(), false, null, true), new HashSet<ByteSequence>(), false);
       while (rowIterator.hasTop()) {
         sum2 += Integer.parseInt(rowIterator.getTopValue().toString());
         rowIterator.next();
       }
-      
+
       return sum == 2 && sum2 == 2;
     }
-    
+
   }
 
   public static class RowZeroOrOneFilter extends RowFilter {
     private static final Set<String> passRows = new HashSet<String>(Arrays.asList("0", "1"));
+
     @Override
     public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
       return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
@@ -90,6 +102,7 @@ public class RowFilterTest extends TestCase {
 
   public static class RowOneOrTwoFilter extends RowFilter {
     private static final Set<String> passRows = new HashSet<String>(Arrays.asList("1", "2"));
+
     @Override
     public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
       return rowIterator.hasTop() && passRows.contains(rowIterator.getTopKey().getRow().toString());
@@ -151,14 +164,36 @@ public class RowFilterTest extends TestCase {
     return mutations;
   }
 
+  public TreeMap<Key,Value> createKeyValues() {
+    List<Mutation> mutations = createMutations();
+    TreeMap<Key,Value> keyValues = new TreeMap<Key,Value>();
+
+    final Text cf = new Text(), cq = new Text();
+    for (Mutation m : mutations) {
+      final Text row = new Text(m.getRow());
+      for (ColumnUpdate update : m.getUpdates()) {
+        cf.set(update.getColumnFamily());
+        cq.set(update.getColumnQualifier());
+
+        Key k = new Key(row, cf, cq);
+        Value v = new Value(update.getValue());
+
+        keyValues.put(k, v);
+      }
+    }
+
+    return keyValues;
+  }
+
+  @Test
   public void test1() throws Exception {
     MockInstance instance = new MockInstance("rft1");
     Connector conn = instance.getConnector("", new PasswordToken(""));
-    
+
     conn.tableOperations().create("table1");
     BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
-    
-    for (Mutation m: createMutations()) {
+
+    for (Mutation m : createMutations()) {
       bw.addMutation(m);
     }
     IteratorSetting is = new IteratorSetting(40, SummingRowFilter.class);
@@ -166,26 +201,26 @@ public class RowFilterTest extends TestCase {
 
     Scanner scanner = conn.createScanner("table1", Constants.NO_AUTHS);
     assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(scanner));
-    
+
     scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
     assertEquals(new HashSet<String>(Arrays.asList("1", "3")), getRows(scanner));
-    
+
     scanner.clearColumns();
     scanner.fetchColumn(new Text("cf1"), new Text("cq1"));
     assertEquals(new HashSet<String>(), getRows(scanner));
-    
+
     scanner.setRange(new Range("0", "4"));
     scanner.clearColumns();
     assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(scanner));
-    
+
     scanner.setRange(new Range("2"));
     scanner.clearColumns();
     assertEquals(new HashSet<String>(Arrays.asList("2")), getRows(scanner));
-    
+
     scanner.setRange(new Range("4"));
     scanner.clearColumns();
     assertEquals(new HashSet<String>(), getRows(scanner));
-    
+
     scanner.setRange(new Range("4"));
     scanner.clearColumns();
     scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
@@ -194,40 +229,84 @@ public class RowFilterTest extends TestCase {
 
   }
 
+  @Test
   public void testChainedRowFilters() throws Exception {
     MockInstance instance = new MockInstance("rft1");
     Connector conn = instance.getConnector("", new PasswordToken(""));
 
     conn.tableOperations().create("chained_row_filters");
     BatchWriter bw = conn.createBatchWriter("chained_row_filters", new BatchWriterConfig());
-    for (Mutation m: createMutations()) {
+    for (Mutation m : createMutations()) {
       bw.addMutation(m);
     }
-    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1",
-        TrueFilter.class));
-    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2",
-        TrueFilter.class));
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1", TrueFilter.class));
+    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2", TrueFilter.class));
     Scanner scanner = conn.createScanner("chained_row_filters", Constants.NO_AUTHS);
     assertEquals(new HashSet<String>(Arrays.asList("0", "1", "2", "3", "4")), getRows(scanner));
   }
 
+  @Test
   public void testFilterConjunction() throws Exception {
     MockInstance instance = new MockInstance("rft1");
     Connector conn = instance.getConnector("", new PasswordToken(""));
 
     conn.tableOperations().create("filter_conjunction");
     BatchWriter bw = conn.createBatchWriter("filter_conjunction", new BatchWriterConfig());
-    for (Mutation m: createMutations()) {
+    for (Mutation m : createMutations()) {
       bw.addMutation(m);
     }
-    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne",
-        RowZeroOrOneFilter.class));
-    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo",
-        RowOneOrTwoFilter.class));
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne", RowZeroOrOneFilter.class));
+    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo", RowOneOrTwoFilter.class));
     Scanner scanner = conn.createScanner("filter_conjunction", Constants.NO_AUTHS);
     assertEquals(new HashSet<String>(Arrays.asList("1")), getRows(scanner));
   }
 
+  @Test
+  public void deepCopyCopiesTheSource() throws Exception {
+    SortedMapIterator source = new SortedMapIterator(createKeyValues());
+
+    RowFilter filter = new RowZeroOrOneFilter();
+    filter.init(source, Collections.<String,String> emptyMap(), new DefaultIteratorEnvironment());
+
+    filter.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+
+    // Save off the first key and value
+    Key firstKey = filter.getTopKey();
+    Value firstValue = filter.getTopValue();
+
+    // Assert that the row is valid given our filter
+    assertEquals("0", firstKey.getRow().toString());
+
+    // Read some extra data, just making sure it's all valid
+    Key lastKeyRead = null;
+    for (int i = 0; i < 5; i++) {
+      filter.next();
+      lastKeyRead = filter.getTopKey();
+      assertEquals("0", lastKeyRead.getRow().toString());
+    }
+
+    // Make a copy of the original RowFilter
+    RowFilter copy = (RowFilter) filter.deepCopy(new DefaultIteratorEnvironment());
+
+    // Because it's a copy, we should be able to safely seek this one without affecting the original
+    copy.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+
+    assertTrue("deepCopy'ed RowFilter did not have a top key", copy.hasTop());
+
+    Key firstKeyFromCopy = copy.getTopKey();
+    Value firstValueFromCopy = copy.getTopValue();
+
+    // Verify that we got the same first k-v pair we did earlier
+    assertEquals(firstKey, firstKeyFromCopy);
+    assertEquals(firstValue, firstValueFromCopy);
+
+    filter.next();
+    Key finalKeyRead = filter.getTopKey();
+
+    // Make sure we got a Key that was greater than the last Key we read from the original RowFilter
+    assertTrue("Expected next key read to be greater than the previous after deepCopy", lastKeyRead.compareTo(finalKeyRead) < 0);
+  }
+
   private HashSet<String> getRows(Scanner scanner) {
     HashSet<String> rows = new HashSet<String>();
     for (Entry<Key,Value> entry : scanner) {