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:34 UTC

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

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