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:33 UTC
[04/10] git commit: ACCUMULO-2501 Add deepCopy to RowFilter
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) {