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) {