From c5810f19a6717dc5ee7f1cf85feada6899c808a2 Mon Sep 17 00:00:00 2001 From: openinx Date: Thu, 20 Dec 2018 21:04:10 +0800 Subject: [PATCH] HBASE-21620 Problem in scan query when using more than one column prefix filter in some cases Signed-off-by: Guanghao Zhang Signed-off-by: Michael Stack Signed-off-by: Allan Yang --- .../hadoop/hbase/filter/FilterListWithOR.java | 65 ++++++++++--------- .../hbase/regionserver/StoreScanner.java | 2 +- .../hadoop/hbase/filter/TestFilterList.java | 62 ++++++++++++++++-- .../hbase/filter/TestFilterListOnMini.java | 50 +++++++++++++- 4 files changed, 140 insertions(+), 39 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java index 064dd8387be..7c32169e195 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java @@ -82,30 +82,40 @@ public class FilterListWithOR extends FilterListBase { * next family for RegionScanner, INCLUDE_AND_NEXT_ROW is the same. so we should pass current cell * to the filter, if row mismatch or row match but column family mismatch. (HBASE-18368) * @see org.apache.hadoop.hbase.filter.Filter.ReturnCode + * @param subFilter which sub-filter to calculate the return code by using previous cell and + * previous return code. + * @param prevCell the previous cell passed to given sub-filter. + * @param currentCell the current cell which will pass to given sub-filter. + * @param prevCode the previous return code for given sub-filter. + * @return return code calculated by using previous cell and previous return code. null means can + * not decide which return code should return, so we will pass the currentCell to + * subFilter for getting currentCell's return code, and it won't impact the sub-filter's + * internal states. */ - private boolean shouldPassCurrentCellToFilter(Cell prevCell, Cell currentCell, - ReturnCode prevCode) throws IOException { + private ReturnCode calculateReturnCodeByPrevCellAndRC(Filter subFilter, Cell currentCell, + Cell prevCell, ReturnCode prevCode) throws IOException { if (prevCell == null || prevCode == null) { - return true; + return null; } switch (prevCode) { case INCLUDE: case SKIP: - return true; + return null; case SEEK_NEXT_USING_HINT: - Cell nextHintCell = getNextCellHint(prevCell); - return nextHintCell == null || this.compareCell(currentCell, nextHintCell) >= 0; + Cell nextHintCell = subFilter.getNextCellHint(prevCell); + return nextHintCell != null && compareCell(currentCell, nextHintCell) < 0 + ? ReturnCode.SEEK_NEXT_USING_HINT : null; case NEXT_COL: case INCLUDE_AND_NEXT_COL: - // Once row changed, reset() will clear prevCells, so we need not to compare their rows - // because rows are the same here. - return !CellUtil.matchingColumn(prevCell, currentCell); + // Once row changed, reset() will clear prevCells, so we need not to compare their rows + // because rows are the same here. + return CellUtil.matchingColumn(prevCell, currentCell) ? ReturnCode.NEXT_COL : null; case NEXT_ROW: case INCLUDE_AND_SEEK_NEXT_ROW: - // As described above, rows are definitely the same, so we only compare the family. - return !CellUtil.matchingFamily(prevCell, currentCell); + // As described above, rows are definitely the same, so we only compare the family. + return CellUtil.matchingFamily(prevCell, currentCell) ? ReturnCode.NEXT_ROW : null; default: - throw new IllegalStateException("Received code is not valid."); + throw new IllegalStateException("Received code is not valid."); } } @@ -239,7 +249,7 @@ public class FilterListWithOR extends FilterListBase { private void updatePrevCellList(int index, Cell currentCell, ReturnCode currentRC) { if (currentCell == null || currentRC == ReturnCode.INCLUDE || currentRC == 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 + // corresponding sub-filter(need not test calculateReturnCodeByPrevCellAndRC() method), So we // need not save current cell to prevCellList for saving heap memory. prevCellList.set(index, null); } else { @@ -253,28 +263,27 @@ public class FilterListWithOR extends FilterListBase { return ReturnCode.INCLUDE; } ReturnCode rc = null; - boolean everyFilterReturnHint = true; for (int i = 0, n = filters.size(); i < n; i++) { Filter filter = filters.get(i); subFiltersIncludedCell.set(i, false); Cell prevCell = this.prevCellList.get(i); ReturnCode prevCode = this.prevFilterRCList.get(i); - if (filter.filterAllRemaining() || !shouldPassCurrentCellToFilter(prevCell, c, prevCode)) { - everyFilterReturnHint = false; + if (filter.filterAllRemaining()) { continue; } - - ReturnCode localRC = filter.filterCell(c); + ReturnCode localRC = calculateReturnCodeByPrevCellAndRC(filter, c, prevCell, prevCode); + if (localRC == null) { + // Can not get return code based on previous cell and previous return code. In other words, + // we should pass the current cell to this sub-filter to get the return code, and it won't + // impact the sub-filter's internal state. + localRC = filter.filterCell(c); + } // Update previous return code and previous cell for filter[i]. updatePrevFilterRCList(i, localRC); updatePrevCellList(i, c, localRC); - if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) { - everyFilterReturnHint = false; - } - rc = mergeReturnCode(rc, localRC); // For INCLUDE* case, we need to update the transformed cell. @@ -283,15 +292,9 @@ public class FilterListWithOR extends FilterListBase { subFiltersIncludedCell.set(i, true); } } - - if (everyFilterReturnHint) { - return ReturnCode.SEEK_NEXT_USING_HINT; - } else if (rc == null) { - // Each sub-filter in filter list got true for filterAllRemaining(). - return ReturnCode.SKIP; - } else { - return rc; - } + // Each sub-filter in filter list got true for filterAllRemaining(), if rc is null, so we should + // return SKIP. + return rc == null ? ReturnCode.SKIP : rc; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index 736c08ab039..9a613bb99f5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -690,7 +690,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner case SEEK_NEXT_USING_HINT: Cell nextKV = matcher.getNextKeyHint(cell); - if (nextKV != null) { + if (nextKV != null && comparator.compare(nextKV, cell) > 0) { seekAsDirection(nextKV); NextState stateAfterSeekByHint = needToReturn(outResult); if (stateAfterSeekByHint != null) { 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 5f9515ac4da..96f6853bcb9 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 @@ -43,6 +43,7 @@ import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.mockito.Mockito; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @@ -602,15 +603,15 @@ public class TestFilterList { 3, Bytes.toBytes("value")); assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList01.filterCell(kv1)); - assertEquals(ReturnCode.SKIP, filterList01.filterCell(kv2)); - assertEquals(ReturnCode.SKIP, filterList01.filterCell(kv3)); + assertEquals(ReturnCode.NEXT_COL, filterList01.filterCell(kv2)); + assertEquals(ReturnCode.NEXT_COL, filterList01.filterCell(kv3)); FilterList filterList11 = new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 1)); assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv1)); - assertEquals(ReturnCode.SKIP, filterList11.filterCell(kv2)); - assertEquals(ReturnCode.SKIP, filterList11.filterCell(kv3)); + assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv2)); + assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv3)); } @Test @@ -629,9 +630,9 @@ public class TestFilterList { Bytes.toBytes("value")); assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1)); - assertEquals(ReturnCode.SKIP, filterList.filterCell(kv2)); + assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv2)); assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv3)); - assertEquals(ReturnCode.SKIP, filterList.filterCell(kv4)); + assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv4)); } private static class MockFilter extends FilterBase { @@ -1019,5 +1020,54 @@ public class TestFilterList { Assert.assertEquals(true, filter2.getTransformed()); Assert.assertEquals(true, filter3.getTransformed()); } + + @Test + public void testFilterListWithORWhenPassingCellMismatchPreviousRC() throws IOException { + // Mainly test FilterListWithOR#calculateReturnCodeByPrevCellAndRC method with two sub-filters. + KeyValue kv1 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("a"), + 100, Bytes.toBytes("value")); + KeyValue kv2 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 99, + Bytes.toBytes("value")); + KeyValue kv3 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("b"), 1, + Bytes.toBytes("value")); + KeyValue kv4 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fan"), Bytes.toBytes("a"), 1, + Bytes.toBytes("value")); + Filter subFilter1 = Mockito.mock(FilterBase.class); + Mockito.when(subFilter1.filterCell(kv1)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL); + Mockito.when(subFilter1.filterCell(kv2)).thenReturn(ReturnCode.NEXT_COL); + Mockito.when(subFilter1.filterCell(kv3)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL); + Mockito.when(subFilter1.filterCell(kv4)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL); + + Filter subFilter2 = Mockito.mock(FilterBase.class); + Mockito.when(subFilter2.filterCell(kv1)).thenReturn(ReturnCode.SKIP); + Mockito.when(subFilter2.filterCell(kv2)).thenReturn(ReturnCode.NEXT_ROW); + Mockito.when(subFilter2.filterCell(kv3)).thenReturn(ReturnCode.NEXT_ROW); + Mockito.when(subFilter2.filterCell(kv4)).thenReturn(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW); + + Filter filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2); + Assert.assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1)); + Assert.assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv2)); + Assert.assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv3)); + Assert.assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv4)); + + // One sub-filter will filterAllRemaining but other sub-filter will return SEEK_HINT + subFilter1 = Mockito.mock(FilterBase.class); + Mockito.when(subFilter1.filterAllRemaining()).thenReturn(true); + Mockito.when(subFilter1.filterCell(kv1)).thenReturn(ReturnCode.NEXT_ROW); + + subFilter2 = Mockito.mock(FilterBase.class); + Mockito.when(subFilter2.filterCell(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT); + filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2); + Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1)); + + // Two sub-filter returns SEEK_NEXT_USING_HINT, then we should return SEEK_NEXT_USING_HINT. + subFilter1 = Mockito.mock(FilterBase.class); + Mockito.when(subFilter1.filterCell(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT); + + subFilter2 = Mockito.mock(FilterBase.class); + Mockito.when(subFilter2.filterCell(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT); + filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2); + Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1)); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java index 7967e0b5cdb..426652239a0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.filter.FilterList.Operator; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -28,7 +29,6 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -90,4 +90,52 @@ public class TestFilterListOnMini { Assert.assertEquals(2, rr.size()); } } + + /** + * Test case for HBASE-21620 + */ + @Test + public void testColumnPrefixFilterConcatWithOR() throws Exception { + TableName tn = TableName.valueOf(name.getMethodName()); + byte[] cf1 = Bytes.toBytes("f1"); + byte[] row = Bytes.toBytes("row"); + byte[] value = Bytes.toBytes("value"); + String[] columns = new String[]{ + "1544768273917010001_lt", + "1544768273917010001_w_1", + "1544768723910010001_ca_1", + "1544768723910010001_lt", + "1544768723910010001_ut_1", + "1544768723910010001_w_5", + "1544769779710010001_lt", + "1544769779710010001_w_5", + "1544769883529010001_lt", + "1544769883529010001_w_5", + "1544769915805010001_lt", + "1544769915805010001_w_5", + "1544779883529010001_lt", + "1544770422942010001_lt", + "1544770422942010001_w_5" + }; + Table table = TEST_UTIL.createTable(tn, cf1); + for (int i = 0; i < columns.length; i++) { + Put put = new Put(row).addColumn(cf1, Bytes.toBytes(columns[i]), value); + table.put(put); + } + Scan scan = new Scan(); + scan.withStartRow(row).withStopRow(row, true) + .setFilter(new FilterList(Operator.MUST_PASS_ONE, + new ColumnPrefixFilter(Bytes.toBytes("1544770422942010001_")), + new ColumnPrefixFilter(Bytes.toBytes("1544769883529010001_")))); + ResultScanner scanner = table.getScanner(scan); + Result result; + int resultCount = 0; + int cellCount = 0; + while ((result = scanner.next()) != null) { + cellCount += result.listCells().size(); + resultCount++; + } + Assert.assertEquals(resultCount, 1); + Assert.assertEquals(cellCount, 4); + } }