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/13 06:55:31 UTC

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

Repository: hbase
Updated Branches:
  refs/heads/branch-1 650ef5cf5 -> c71da858a


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/c71da858
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c71da858
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c71da858

Branch: refs/heads/branch-1
Commit: c71da858ada94e1b93065f0b7caf3558942bc4da
Parents: 650ef5c
Author: huzheng <op...@gmail.com>
Authored: Thu Jun 8 13:42:23 2017 +0800
Committer: tedyu <yu...@gmail.com>
Committed: Mon Jun 12 23:55:25 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/filter/FilterList.java  |  82 ++++++++++++-
 .../org/apache/hadoop/hbase/KeyValueUtil.java   |  15 +++
 .../hadoop/hbase/filter/TestFilterList.java     | 117 ++++++++++++++++++-
 3 files changed, 211 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c71da858/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 be22e5d..ddbd4a7 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
@@ -21,8 +21,10 @@ package org.apache.hadoop.hbase.filter;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
@@ -69,6 +71,14 @@ final public class FilterList extends Filter {
   private List<Filter> filters = new ArrayList<Filter>();
   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 referenceKV = null;
 
@@ -93,6 +103,7 @@ final public class FilterList extends Filter {
     } else {
       this.filters = new ArrayList<Filter>(rowFilters);
     }
+    initPrevListForMustPassOne(rowFilters.size());
   }
 
   /**
@@ -102,6 +113,7 @@ final public class FilterList extends Filter {
    */
   public FilterList(final Filter... rowFilters) {
     this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
+    initPrevListForMustPassOne(filters.size());
   }
 
   /**
@@ -111,6 +123,7 @@ final public class FilterList extends Filter {
    */
   public FilterList(final Operator operator) {
     this.operator = operator;
+    initPrevListForMustPassOne(filters.size());
   }
 
   /**
@@ -122,6 +135,7 @@ final public class FilterList extends Filter {
   public FilterList(final Operator operator, final List<Filter> rowFilters) {
     this.filters = new ArrayList<Filter>(rowFilters);
     this.operator = operator;
+    initPrevListForMustPassOne(filters.size());
   }
 
   /**
@@ -133,6 +147,18 @@ final public class FilterList extends Filter {
   public FilterList(final Operator operator, final Filter... rowFilters) {
     this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
     this.operator = operator;
+    initPrevListForMustPassOne(filters.size());
+  }
+
+  public void initPrevListForMustPassOne(int size) {
+    if (operator == Operator.MUST_PASS_ONE) {
+      if (this.prevFilterRCList == null) {
+        prevFilterRCList = new ArrayList<ReturnCode>(Collections.nCopies(size, (ReturnCode) null));
+      }
+      if (this.prevCellList == null) {
+        prevCellList = new ArrayList<Cell>(Collections.nCopies(size, (Cell) null));
+      }
+    }
   }
 
   /**
@@ -165,6 +191,10 @@ final public class FilterList extends Filter {
               + this.isReversed());
     }
     this.filters.add(filter);
+    if (operator == Operator.MUST_PASS_ONE) {
+      this.prevFilterRCList.add((ReturnCode) null);
+      this.prevCellList.add((Cell) null);
+    }
   }
 
   @Override
@@ -172,6 +202,10 @@ final public class FilterList extends Filter {
     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;
   }
@@ -242,6 +276,38 @@ final public class FilterList extends Filter {
     return KeyValueUtil.ensureKeyValue(this.transformedKV);
   }
 
+  /**
+   * 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 || KeyValue.COMPARATOR.compare(currentCell, nextHintCell) >= 0;
+    case NEXT_COL:
+    case INCLUDE_AND_NEXT_COL:
+      return !CellUtil.matchingRowColumn(prevCell, currentCell);
+    case NEXT_ROW:
+      return !CellUtil.matchingRow(prevCell, currentCell);
+    default:
+      throw new IllegalStateException("Received code is not valid.");
+    }
+  }
   
   @Override
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
@@ -285,12 +351,24 @@ final public class FilterList extends Filter {
           return code;
         }
       } else if (operator == Operator.MUST_PASS_ONE) {
-        if (filter.filterAllRemaining()) {
+        Cell prevCell = this.prevCellList.get(i);
+        if (filter.filterAllRemaining() || !shouldPassCurrentCellToFilter(prevCell, v, i)) {
           seenNonHintReturnCode = true;
           continue;
         }
 
         ReturnCode localRC = filter.filterKeyValue(v);
+        // Update previous cell and return code we encountered.
+        prevFilterRCList.set(i, localRC);
+        if (v == null || localRC == ReturnCode.INCLUDE || localRC == ReturnCode.SKIP) {
+          // If previous return code is INCLUDE or SKIP, we should always pass the next cell to the
+          // corresponding sub-filter(need not test shouldPassCurrentCellToFilter() method), So we
+          // need not save current cell to prevCellList for saving heap memory.
+          prevCellList.set(i, null);
+        } else {
+          prevCellList.set(i, KeyValueUtil.toNewKeyCell(v));
+        }
+
         if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) {
           seenNonHintReturnCode = true;
         }
@@ -444,7 +522,7 @@ final public class FilterList extends Filter {
   public Cell getNextCellHint(Cell currentKV) throws IOException {
     Cell keyHint = null;
     if (operator == Operator.MUST_PASS_ALL) {
-      keyHint = seekHintFilter.getNextCellHint(currentKV);
+      if (seekHintFilter != null) keyHint = seekHintFilter.getNextCellHint(currentKV);
       return keyHint;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c71da858/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
index 6d502fd..12b19f6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
@@ -117,6 +117,21 @@ public class KeyValueUtil {
     return buffer;
   }
 
+  /**
+   * Copies the key to a new KeyValue
+   * @param cell
+   * @return the KeyValue that consists only the key part of the incoming cell
+   */
+  public static KeyValue toNewKeyCell(final Cell cell) {
+    byte[] bytes = new byte[keyLength(cell)];
+    appendKeyTo(cell, bytes, 0);
+    KeyValue kv = new KeyValue.KeyOnlyKeyValue(bytes, 0, bytes.length);
+    // Set the seq id. The new key cell could be used in comparisons so it
+    // is important that it uses the seqid also. If not the comparsion would fail
+    kv.setSequenceId(cell.getSequenceId());
+    return kv;
+  }
+
   public static byte[] copyToNewByteArray(final Cell cell) {
     int v1Length = length(cell);
     byte[] backingBytes = new byte[v1Length];

http://git-wip-us.apache.org/repos/asf/hbase/blob/c71da858/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 6ddc422..994adff 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,7 +25,6 @@ import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNull;
 
@@ -554,5 +553,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);
+  }
 }