You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2017/06/13 12:30:09 UTC
[1/2] hbase git commit: Revert "FilterList with MUST_PASS_ONE may
lead to redundant cells returned" miss issue number
Repository: hbase
Updated Branches:
refs/heads/branch-1 c71da858a -> 256fc6300
Revert "FilterList with MUST_PASS_ONE may lead to redundant cells returned" miss issue number
This reverts commit c71da858ada94e1b93065f0b7caf3558942bc4da.
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d6e85b05
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d6e85b05
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d6e85b05
Branch: refs/heads/branch-1
Commit: d6e85b0511396b3221cc7f495eaee5bbacc42afd
Parents: c71da85
Author: zhangduo <zh...@apache.org>
Authored: Tue Jun 13 20:27:28 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Jun 13 20:27:28 2017 +0800
----------------------------------------------------------------------
.../apache/hadoop/hbase/filter/FilterList.java | 82 +------------
.../org/apache/hadoop/hbase/KeyValueUtil.java | 15 ---
.../hadoop/hbase/filter/TestFilterList.java | 117 +------------------
3 files changed, 3 insertions(+), 211 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/d6e85b05/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 ddbd4a7..be22e5d 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,10 +21,8 @@ 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;
@@ -71,14 +69,6 @@ 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;
@@ -103,7 +93,6 @@ final public class FilterList extends Filter {
} else {
this.filters = new ArrayList<Filter>(rowFilters);
}
- initPrevListForMustPassOne(rowFilters.size());
}
/**
@@ -113,7 +102,6 @@ final public class FilterList extends Filter {
*/
public FilterList(final Filter... rowFilters) {
this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
- initPrevListForMustPassOne(filters.size());
}
/**
@@ -123,7 +111,6 @@ final public class FilterList extends Filter {
*/
public FilterList(final Operator operator) {
this.operator = operator;
- initPrevListForMustPassOne(filters.size());
}
/**
@@ -135,7 +122,6 @@ 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());
}
/**
@@ -147,18 +133,6 @@ 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));
- }
- }
}
/**
@@ -191,10 +165,6 @@ 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
@@ -202,10 +172,6 @@ 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;
}
@@ -276,38 +242,6 @@ 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",
@@ -351,24 +285,12 @@ final public class FilterList extends Filter {
return code;
}
} else if (operator == Operator.MUST_PASS_ONE) {
- Cell prevCell = this.prevCellList.get(i);
- if (filter.filterAllRemaining() || !shouldPassCurrentCellToFilter(prevCell, v, i)) {
+ if (filter.filterAllRemaining()) {
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;
}
@@ -522,7 +444,7 @@ final public class FilterList extends Filter {
public Cell getNextCellHint(Cell currentKV) throws IOException {
Cell keyHint = null;
if (operator == Operator.MUST_PASS_ALL) {
- if (seekHintFilter != null) keyHint = seekHintFilter.getNextCellHint(currentKV);
+ keyHint = seekHintFilter.getNextCellHint(currentKV);
return keyHint;
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/d6e85b05/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 12b19f6..6d502fd 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,21 +117,6 @@ 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/d6e85b05/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 994adff..6ddc422 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.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;
@@ -553,121 +554,5 @@ 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);
- }
}
[2/2] hbase git commit: HBASE-17678 FilterList with MUST_PASS_ONE may
lead to redundant cells returned
Posted by zh...@apache.org.
HBASE-17678 FilterList with MUST_PASS_ONE may lead to redundant cells returned
Signed-off-by: tedyu <yu...@gmail.com>
Signed-off-by: zhangduo <zh...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/256fc630
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/256fc630
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/256fc630
Branch: refs/heads/branch-1
Commit: 256fc63007aecb63028b71ad1383d896f11db701
Parents: d6e85b0
Author: huzheng <op...@gmail.com>
Authored: Thu Jun 8 13:42:23 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Jun 13 20:28:04 2017 +0800
----------------------------------------------------------------------
.../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/256fc630/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/256fc630/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/256fc630/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);
+ }
}