You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2024/01/26 20:17:27 UTC

(accumulo) branch 2.1 updated: Fixes RowFilter bug with deep copy (#4197)

This is an automated email from the ASF dual-hosted git repository.

kturner pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/2.1 by this push:
     new 5401dedda0 Fixes RowFilter bug with deep copy (#4197)
5401dedda0 is described below

commit 5401dedda08479042d78b52db25f6ee741b936cb
Author: Keith Turner <kt...@apache.org>
AuthorDate: Fri Jan 26 15:17:21 2024 -0500

    Fixes RowFilter bug with deep copy (#4197)
    
    If a RowFilter implemented init and used the options, then these options
    would be lost if the RowFilter was deep copied.  When RowFilters are
    chained the 2nd RowFilter in the chain will deep copy the first.
    This commit fixes the bug by saving options and calling init with them
    in deep copy.
---
 .../accumulo/core/iterators/user/RowFilter.java    |  4 +-
 .../core/iterators/user/RowFilterTest.java         | 89 ++++++++++++++++++++++
 2 files changed, 92 insertions(+), 1 deletion(-)

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 932195501d..54d8b0978d 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
@@ -60,6 +60,7 @@ public abstract class RowFilter extends WrappingIterator {
   private boolean inclusive;
   private Range range;
   private boolean hasTop;
+  private Map<String,String> options;
 
   private static class RowIterator extends WrappingIterator {
     private Range rowRange;
@@ -148,6 +149,7 @@ public abstract class RowFilter extends WrappingIterator {
       IteratorEnvironment env) throws IOException {
     super.init(source, options, env);
     this.decisionIterator = new RowIterator(source.deepCopy(env));
+    this.options = Map.copyOf(options);
   }
 
   @Override
@@ -155,10 +157,10 @@ public abstract class RowFilter extends WrappingIterator {
     RowFilter newInstance;
     try {
       newInstance = getClass().getDeclaredConstructor().newInstance();
+      newInstance.init(getSource().deepCopy(env), options, env);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
-    newInstance.setSource(getSource().deepCopy(env));
     newInstance.decisionIterator = new RowIterator(getSource().deepCopy(env));
     return newInstance;
   }
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 81b4dabbeb..a18583efb4 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
@@ -19,13 +19,16 @@
 package org.apache.accumulo.core.iterators.user;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
 
@@ -37,6 +40,7 @@ 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.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.ColumnFamilySkippingIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.SortedMapIterator;
@@ -108,6 +112,24 @@ public class RowFilterTest {
     }
   }
 
+  public static class ColFamFilter extends RowFilter {
+    private Set<ByteSequence> families;
+
+    @Override
+    public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
+        IteratorEnvironment env) throws IOException {
+      super.init(source, options, env);
+      this.families = Set.of(new ArrayByteSequence(options.get("family")));
+    }
+
+    @Override
+    public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException {
+      rowIterator.seek(new Range(), families, true);
+      return rowIterator.hasTop();
+    }
+
+  }
+
   public List<Mutation> createMutations() {
     List<Mutation> mutations = new LinkedList<>();
     Mutation m = new Mutation("0");
@@ -286,6 +308,73 @@ public class RowFilterTest {
         "Expected next key read to be greater than the previous after deepCopy");
   }
 
+  // This tests a chaining RowFitler that have an init method and deep copying the chain. There was
+  // a bug where if the row filter called init it was not called on deep copy.
+  @Test
+  public void testDeepCopyChainedIteratorWithInit() throws Exception {
+    TreeMap<Key,Value> data = new TreeMap<>();
+    data.put(new Key("1", "f1", "q1"), new Value("1"));
+    data.put(new Key("1", "f2", "q1"), new Value("2"));
+    data.put(new Key("2", "f1", "q1"), new Value("3"));
+    data.put(new Key("2", "f2", "q1"), new Value("4"));
+    data.put(new Key("2", "f3", "q1"), new Value("5"));
+    data.put(new Key("3", "f1", "q1"), new Value("6"));
+    data.put(new Key("3", "f2", "q1"), new Value("7"));
+    data.put(new Key("3", "f3", "q1"), new Value("8"));
+    data.put(new Key("3", "f4", "q1"), new Value("9"));
+    data.put(new Key("4", "f3", "q1"), new Value("0"));
+    data.put(new Key("5", "f1", "q1"), new Value("a"));
+    data.put(new Key("5", "f3", "q1"), new Value("b"));
+
+    var source = new ColumnFamilySkippingIterator(new SortedMapIterator(data));
+
+    RowFilter filter1 = new ColFamFilter();
+    filter1.init(source, Map.of("family", "f1"), null);
+
+    // The chain of filter1 and filter2 should only return rows that have families 'f1' and 'f3'.
+    RowFilter filter2 = new ColFamFilter();
+    filter2.init(filter1, Map.of("family", "f3"), null);
+
+    // Deep copy of filter2 should be able to independently iterator over the same data.
+    var filter3 = (RowFilter) filter2.deepCopy(null);
+
+    for (int i = 0; i < 3; i++) {
+      filter2.seek(new Range(), Set.of(), false);
+      filter3.seek(new Range(), Set.of(), false);
+
+      Iterator<Key> expectedKeys2 = data.keySet().stream().filter(k -> {
+        var row = k.getRowData().toString();
+        return row.equals("2") || row.equals("3") || row.equals("5");
+      }).iterator();
+
+      Iterator<Key> expectedKeys3 = data.keySet().stream().filter(k -> {
+        var row = k.getRowData().toString();
+        return row.equals("2") || row.equals("3") || row.equals("5");
+      }).iterator();
+
+      // advance one iterator and not the other so they are on different keys in the loop
+      assertEquals(expectedKeys2.next(), filter2.getTopKey());
+      filter2.next();
+
+      while (expectedKeys2.hasNext() || filter2.hasTop() || expectedKeys3.hasNext()
+          || filter3.hasTop()) {
+        if (expectedKeys2.hasNext()) {
+          assertEquals(expectedKeys2.next(), filter2.getTopKey());
+          filter2.next();
+        }
+        if (expectedKeys3.hasNext()) {
+          assertEquals(expectedKeys3.next(), filter3.getTopKey());
+          filter3.next();
+        }
+      }
+
+      assertFalse(expectedKeys2.hasNext());
+      assertFalse(filter2.hasTop());
+      assertFalse(expectedKeys3.hasNext());
+      assertFalse(filter3.hasTop());
+    }
+  }
+
   private HashSet<String> getRows(RowFilter filter) throws IOException {
     HashSet<String> rows = new HashSet<>();
     while (filter.hasTop()) {