Index: src/test/org/apache/hadoop/hbase/filter/TestFilter.java =================================================================== --- src/test/org/apache/hadoop/hbase/filter/TestFilter.java (revision dc1f2b71da10a01d1fad397e799bb3fa190c86e9) +++ src/test/org/apache/hadoop/hbase/filter/TestFilter.java Wed Feb 24 21:05:15 EST 2010 @@ -4,7 +4,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import junit.framework.Assert; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseTestCase; @@ -253,7 +255,94 @@ verifyScanFull(s, Arrays.copyOf(expectedKVs, 6)); } - + + /** + * Tests the the {@link WhileMatchFilter} works in combination with a + * {@link Filter} that uses the + * {@link PageFilter#filterRow()} method. + * + * See HBASE-2258. + * + * @throws Exception + */ + public void testWhileMatchFilterWithFilterRow() throws Exception { + final int pageSize = 4; + + Scan s = new Scan(); + WhileMatchFilter filter = new WhileMatchFilter(new PageFilter(pageSize)); + s.setFilter(filter); + + InternalScanner scanner = this.region.getScanner(s); + int scannerCounter = 0; + while (true) { + boolean isMoreResults = scanner.next(new ArrayList()); + scannerCounter++; + + if (scannerCounter >= pageSize) { + Assert.assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining()); + } + if (!isMoreResults) { + break; + } + } + Assert.assertEquals("The page filter returned more rows than expected", pageSize, scannerCounter); + } + + /** + * Tests the the {@link WhileMatchFilter} works in combination with a + * {@link Filter} that uses the + * {@link PageFilter#filterRowKey(byte[], int, int)} method. + * + * See HBASE-2258. + * + * @throws Exception + */ + public void testWhileMatchFilterWithFilterRowKey() throws Exception { + Scan s = new Scan(); + String prefix = "testRowOne"; + WhileMatchFilter filter = new WhileMatchFilter(new PrefixFilter(Bytes.toBytes(prefix))); + s.setFilter(filter); + + InternalScanner scanner = this.region.getScanner(s); + while (true) { + ArrayList values = new ArrayList(); + boolean isMoreResults = scanner.next(values); + if (!isMoreResults || !Bytes.toString(values.get(0).getRow()).startsWith(prefix)) { + Assert.assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining()); + } + if (!isMoreResults) { + break; + } + } + } + + /** + * Tests the the {@link WhileMatchFilter} works in combination with a + * {@link Filter} that uses the + * {@link PageFilter#filterKeyValue(org.apache.hadoop.hbase.KeyValue)} method. + * + * See HBASE-2258. + * + * @throws Exception + */ + public void testWhileMatchFilterWithFilterKeyValue() throws Exception { + Scan s = new Scan(); + WhileMatchFilter filter = new WhileMatchFilter( + new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOp.EQUAL, Bytes.toBytes("foo")) + ); + s.setFilter(filter); + + InternalScanner scanner = this.region.getScanner(s); + while (true) { + ArrayList values = new ArrayList(); + boolean isMoreResults = scanner.next(values); + Assert.assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining()); + if (!isMoreResults) { + break; + } + } + } + public void testInclusiveStopFilter() throws IOException { // Grab rows from group one Index: src/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java =================================================================== --- src/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java (revision dc1f2b71da10a01d1fad397e799bb3fa190c86e9) +++ src/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java Wed Feb 24 21:16:15 EST 2010 @@ -27,7 +27,11 @@ import java.io.DataInput; /** - * A wrapper filter that filters everything after the first filtered row. + * A wrapper filter that filters everything after the first time false is returned + * from any wrapped filters {@link Filter#filterRowKey(byte[], int, int)}, + * {@link Filter#filterKeyValue(org.apache.hadoop.hbase.KeyValue)}, + * {@link org.apache.hadoop.hbase.filter.Filter#filterRow()} or + * {@link org.apache.hadoop.hbase.filter.Filter#filterAllRemaining()}. */ public class WhileMatchFilter implements Filter { private boolean filterAllRemaining = false; @@ -49,15 +53,16 @@ filterAllRemaining = filterAllRemaining || value; } - public boolean filterRowKey(byte[] buffer, int offset, int length) { - changeFAR(filter.filterRowKey(buffer, offset, length)); - return filterAllRemaining(); - } - public boolean filterAllRemaining() { return this.filterAllRemaining || this.filter.filterAllRemaining(); } + public boolean filterRowKey(byte[] buffer, int offset, int length) { + boolean value = filter.filterRowKey(buffer, offset, length); + changeFAR(value); + return value; + } + public ReturnCode filterKeyValue(KeyValue v) { ReturnCode c = filter.filterKeyValue(v); changeFAR(c != ReturnCode.INCLUDE); @@ -65,7 +70,9 @@ } public boolean filterRow() { - return false; + boolean filterRow = this.filter.filterRow(); + changeFAR(filterRow); + return filterRow; } public void write(DataOutput out) throws IOException {