You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2017/06/07 04:08:17 UTC

hbase git commit: HBASE-17678 FilterList with MUST_PASS_ONE may lead to redundant cells returned

Repository: hbase
Updated Branches:
  refs/heads/master 2f1923a82 -> 0d0c33040


HBASE-17678 FilterList with MUST_PASS_ONE may lead to redundant cells returned

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/master
Commit: 0d0c330401ade938bf934aafd79ec23705edcc60
Parents: 2f1923a
Author: huzheng <op...@gmail.com>
Authored: Sat May 27 16:58:00 2017 +0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Jun 6 21:08:12 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/filter/FilterList.java  |  76 +++++++++++-
 .../hadoop/hbase/filter/TestFilterList.java     | 117 +++++++++++++++++++
 2 files changed, 191 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0d0c3304/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
index 0742b22..985cb16 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
@@ -66,6 +66,14 @@ final public class FilterList extends FilterBase {
   private final List<Filter> filters;
   private Filter seekHintFilter = null;
 
+  /**
+   * Save previous return code and previous cell for every filter in filter list. For MUST_PASS_ONE,
+   * we use the previous return code to decide whether we should pass current cell encountered to
+   * the filter. For MUST_PASS_ALL, the two list are meaningless.
+   */
+  private List<ReturnCode> prevFilterRCList = null;
+  private List<Cell> prevCellList = null;
+
   /** Reference Cell used by {@link #transformCell(Cell)} for validation purpose. */
   private Cell referenceCell = null;
 
@@ -87,6 +95,7 @@ final public class FilterList extends FilterBase {
   public FilterList(final List<Filter> rowFilters) {
     reversed = getReversed(rowFilters, reversed);
     this.filters = new ArrayList<>(rowFilters);
+    initPrevListForMustPassOne(rowFilters.size());
   }
 
   /**
@@ -106,6 +115,7 @@ final public class FilterList extends FilterBase {
   public FilterList(final Operator operator) {
     this.operator = operator;
     this.filters = new ArrayList<>();
+    initPrevListForMustPassOne(filters.size());
   }
 
   /**
@@ -117,6 +127,7 @@ final public class FilterList extends FilterBase {
   public FilterList(final Operator operator, final List<Filter> rowFilters) {
     this(rowFilters);
     this.operator = operator;
+    initPrevListForMustPassOne(rowFilters.size());
   }
 
   /**
@@ -128,8 +139,21 @@ final public class FilterList extends FilterBase {
   public FilterList(final Operator operator, final Filter... rowFilters) {
     this(rowFilters);
     this.operator = operator;
+    initPrevListForMustPassOne(rowFilters.length);
+  }
+
+  public void initPrevListForMustPassOne(int size) {
+    if (operator == Operator.MUST_PASS_ONE) {
+      if (this.prevCellList == null) {
+        prevFilterRCList = new ArrayList<>(Collections.nCopies(size, null));
+      }
+      if (this.prevCellList == null) {
+        prevCellList = new ArrayList<>(Collections.nCopies(size, null));
+      }
+    }
   }
 
+
   /**
    * Get the operator.
    *
@@ -184,6 +208,10 @@ final public class FilterList extends FilterBase {
   public void addFilter(List<Filter> filters) {
     checkReversed(filters, isReversed());
     this.filters.addAll(filters);
+    if (operator == Operator.MUST_PASS_ONE) {
+      this.prevFilterRCList.addAll(Collections.nCopies(filters.size(), null));
+      this.prevCellList.addAll(Collections.nCopies(filters.size(), null));
+    }
   }
 
   /**
@@ -200,6 +228,10 @@ final public class FilterList extends FilterBase {
     int listize = filters.size();
     for (int i = 0; i < listize; i++) {
       filters.get(i).reset();
+      if (operator == Operator.MUST_PASS_ONE) {
+        prevFilterRCList.set(i, null);
+        prevCellList.set(i, null);
+      }
     }
     seekHintFilter = null;
   }
@@ -282,6 +314,41 @@ final public class FilterList extends FilterBase {
     return this.transformedCell;
   }
 
+  /**
+   * For MUST_PASS_ONE, we cannot make sure that when filter-A in filter list return NEXT_COL then
+   * the next cell passing to filterList will be the first cell in next column, because if filter-B
+   * in filter list return SKIP, then the filter list will return SKIP. In this case, we should pass
+   * the cell following the previous cell, and it's possible that the next cell has the same column
+   * as the previous cell even if filter-A has NEXT_COL returned for the previous cell. So we should
+   * save the previous cell and the return code list when checking previous cell for every filter in
+   * filter list, and verify if currentCell fit the previous return code, if fit then pass the currentCell
+   * to the corresponding filter. (HBASE-17678)
+   */
+  private boolean shouldPassCurrentCellToFilter(Cell prevCell, Cell currentCell, int filterIdx)
+      throws IOException {
+    ReturnCode prevCode = this.prevFilterRCList.get(filterIdx);
+    if (prevCell == null || prevCode == null) {
+      return true;
+    }
+    switch (prevCode) {
+    case INCLUDE:
+    case SKIP:
+      return true;
+    case SEEK_NEXT_USING_HINT:
+      Cell nextHintCell = getNextCellHint(prevCell);
+      return nextHintCell == null
+          || CellComparator.COMPARATOR.compare(currentCell, nextHintCell) >= 0;
+    case NEXT_COL:
+    case INCLUDE_AND_NEXT_COL:
+      return !CellUtil.matchingRowColumn(prevCell, currentCell);
+    case NEXT_ROW:
+    case INCLUDE_AND_SEEK_NEXT_ROW:
+      return !CellUtil.matchingRows(prevCell, currentCell);
+    default:
+      throw new IllegalStateException("Received code is not valid.");
+    }
+  }
+
   @Override
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
     justification="Intentional")
@@ -327,12 +394,17 @@ final public class FilterList extends FilterBase {
           return code;
         }
       } else if (operator == Operator.MUST_PASS_ONE) {
-        if (filter.filterAllRemaining()) {
+        Cell prevCell = this.prevCellList.get(i);
+        if (filter.filterAllRemaining() || !shouldPassCurrentCellToFilter(prevCell, c, i)) {
           seenNonHintReturnCode = true;
           continue;
         }
 
         ReturnCode localRC = filter.filterKeyValue(c);
+        // Update previous cell and return code we encountered.
+        prevFilterRCList.set(i, localRC);
+        prevCellList.set(i, c);
+
         if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) {
           seenNonHintReturnCode = true;
         }
@@ -485,7 +557,7 @@ final public class FilterList extends FilterBase {
     }
     Cell keyHint = null;
     if (operator == Operator.MUST_PASS_ALL) {
-      keyHint = seekHintFilter.getNextCellHint(currentCell);
+      if (seekHintFilter != null) keyHint = seekHintFilter.getNextCellHint(currentCell);
       return keyHint;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d0c3304/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
index ad71fcc..cf8a0a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
@@ -25,6 +25,7 @@ import java.util.Arrays;
 import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -593,5 +594,121 @@ public class TestFilterList {
     assertEquals(Filter.ReturnCode.SKIP, flist.filterKeyValue(kvQual3));
   }
 
+  @Test
+  public void testWithMultiVersionsInSameRow() throws Exception {
+    FilterList filterList01 =
+        new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 0));
+
+    KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
+        1, Bytes.toBytes("value"));
+    KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
+        2, Bytes.toBytes("value"));
+    KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
+        3, Bytes.toBytes("value"));
+
+    assertEquals(filterList01.filterKeyValue(kv1), ReturnCode.INCLUDE_AND_NEXT_COL);
+    assertEquals(filterList01.filterKeyValue(kv2), ReturnCode.SKIP);
+    assertEquals(filterList01.filterKeyValue(kv3), ReturnCode.SKIP);
+
+    FilterList filterList11 =
+        new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 1));
+
+    assertEquals(filterList11.filterKeyValue(kv1), ReturnCode.SKIP);
+    assertEquals(filterList11.filterKeyValue(kv2), ReturnCode.SKIP);
+    assertEquals(filterList11.filterKeyValue(kv3), ReturnCode.SKIP);
+  }
+
+  @Test
+  public void testMPONEWithSeekNextUsingHint() throws Exception {
+    byte[] col = Bytes.toBytes("c");
+    FilterList filterList =
+        new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, col));
+
+    KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
+        Bytes.toBytes("value"));
+    KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("b"), 2,
+        Bytes.toBytes("value"));
+    KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 3,
+        Bytes.toBytes("value"));
+    KeyValue kv4 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 4,
+        Bytes.toBytes("value"));
+
+    assertEquals(filterList.filterKeyValue(kv1), ReturnCode.SEEK_NEXT_USING_HINT);
+    assertEquals(filterList.filterKeyValue(kv2), ReturnCode.SKIP);
+    assertEquals(filterList.filterKeyValue(kv3), ReturnCode.INCLUDE_AND_NEXT_COL);
+    assertEquals(filterList.filterKeyValue(kv4), ReturnCode.SKIP);
+  }
+
+  private static class MockFilter extends FilterBase {
+    private ReturnCode targetRetCode;
+    public boolean didCellPassToTheFilter = false;
+
+    public MockFilter(ReturnCode targetRetCode) {
+      this.targetRetCode = targetRetCode;
+    }
+
+    @Override
+    public ReturnCode filterKeyValue(Cell v) throws IOException {
+      this.didCellPassToTheFilter = true;
+      return targetRetCode;
+    }
+  }
+
+  @Test
+  public void testShouldPassCurrentCellToFilter() throws IOException {
+    KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
+        Bytes.toBytes("value"));
+    KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 2,
+        Bytes.toBytes("value"));
+    KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("b"), 3,
+        Bytes.toBytes("value"));
+    KeyValue kv4 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 4,
+        Bytes.toBytes("value"));
+
+    MockFilter mockFilter = new MockFilter(ReturnCode.NEXT_COL);
+    FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
+
+    filter.filterKeyValue(kv1);
+    assertTrue(mockFilter.didCellPassToTheFilter);
+
+    mockFilter.didCellPassToTheFilter = false;
+    filter.filterKeyValue(kv2);
+    assertFalse(mockFilter.didCellPassToTheFilter);
+
+    mockFilter.didCellPassToTheFilter = false;
+    filter.filterKeyValue(kv3);
+    assertTrue(mockFilter.didCellPassToTheFilter);
+
+    mockFilter = new MockFilter(ReturnCode.INCLUDE_AND_NEXT_COL);
+    filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
+
+    filter.filterKeyValue(kv1);
+    assertTrue(mockFilter.didCellPassToTheFilter);
+
+    mockFilter.didCellPassToTheFilter = false;
+    filter.filterKeyValue(kv2);
+    assertFalse(mockFilter.didCellPassToTheFilter);
+
+    mockFilter.didCellPassToTheFilter = false;
+    filter.filterKeyValue(kv3);
+    assertTrue(mockFilter.didCellPassToTheFilter);
+
+    mockFilter = new MockFilter(ReturnCode.NEXT_ROW);
+    filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
+    filter.filterKeyValue(kv1);
+    assertTrue(mockFilter.didCellPassToTheFilter);
+
+    mockFilter.didCellPassToTheFilter = false;
+    filter.filterKeyValue(kv2);
+    assertFalse(mockFilter.didCellPassToTheFilter);
+
+    mockFilter.didCellPassToTheFilter = false;
+    filter.filterKeyValue(kv3);
+    assertFalse(mockFilter.didCellPassToTheFilter);
+
+    mockFilter.didCellPassToTheFilter = false;
+    filter.filterKeyValue(kv4);
+    assertTrue(mockFilter.didCellPassToTheFilter);
+  }
 }