diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 7f8790a..633c262 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -3913,10 +3913,12 @@ public class HRegion implements HeapSize { // , Writable{ if (filter != null && filter.hasFilterRow()) { filter.filterRowCells(results); } - if (isEmptyRow) { + + if (isEmptyRow || filterRow()) { + results.clear(); boolean moreRows = nextRow(currentRow, offset, length); if (!moreRows) return false; - results.clear(); + // This row was totally filtered out, if this is NOT the last row, // we should continue on. Otherwise, nothing else to do. if (!stopRow) continue; @@ -3966,6 +3968,20 @@ public class HRegion implements HeapSize { // , Writable{ } } + /** + * This function is to maintain backward compatibility for 0.94 filters. HBASE-6429 combines + * both filterRow & filterRow(List kvs) functions. While 0.94 code or older, it may + * not implement hasFilterRow as HBase-6429 expects because 0.94 hasFilterRow() only returns + * true when filterRow(List kvs) is overridden not the filterRow(). Therefore, the + * filterRow() will be skipped. + */ + private boolean filterRow() throws IOException { + // when hasFilterRow returns true, filter.filterRow() will be called automatically inside + // filterRowCells(List kvs) so we skip that scenario here. + return filter != null && (!filter.hasFilterRow()) + && filter.filterRow(); + } + private boolean filterRowKey(byte[] row, int offset, short length) throws IOException { return filter != null && filter.filterRowKey(row, offset, length); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index ca5098d..0068779 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -579,6 +579,47 @@ public class TestFilter { assertEquals("The page filter returned more rows than expected", pageSize, scannerCounter); } + + /** + * The following filter simulates a pre-0.96 filter where filterRow() is defined while + * hasFilterRow() returns false + */ + static class OldTestFilter extends FilterBase { + @Override + public byte [] toByteArray() {return null;} + + @Override + public boolean hasFilterRow() { + return false; + } + + @Override + public boolean filterRow() { + // always filter out rows + return true; + } + } + + /** + * The following test is to ensure old(such as hbase0.94) filterRow() can be correctly fired in + * 0.96+ code base. + * + * See HBASE-10366 + * + * @throws Exception + */ + @Test + public void tes94FilterRowCompatibility() throws Exception { + Scan s = new Scan(); + OldTestFilter filter = new OldTestFilter(); + s.setFilter(filter); + + InternalScanner scanner = this.region.getScanner(s); + ArrayList values = new ArrayList(); + scanner.next(values); + assertTrue("All rows should be filtered out", values.isEmpty()); + } + /** * Tests the the {@link WhileMatchFilter} works in combination with a * {@link Filter} that uses the