Index: hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java (revision 1583858) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java (working copy) @@ -119,12 +119,15 @@ /* Will pass both */ byte [] rowkey = Bytes.toBytes("yyyyyyyyy"); + List kvs = new ArrayList(); for (int i = 0; i < MAX_PAGES - 1; i++) { assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length)); KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i)); assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv)); - assertFalse(filterMPONE.filterRow()); + kvs.add(kv); + filterMPONE.filterRow(kvs); + assertFalse(kvs.isEmpty()); } /* Only pass PageFilter */ @@ -133,15 +136,21 @@ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0), Bytes.toBytes(0)); assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv)); - assertFalse(filterMPONE.filterRow()); + kvs.clear(); + kvs.add(kv); + filterMPONE.filterRow(kvs); + assertFalse(kvs.isEmpty()); /* reach MAX_PAGES already, should filter any rows */ rowkey = Bytes.toBytes("yyy"); + kvs.clear(); assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length)); kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0), Bytes.toBytes(0)); assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv)); - assertFalse(filterMPONE.filterRow()); + kvs.add(kv); + filterMPONE.filterRow(kvs); + assertFalse(kvs.isEmpty()); /* We should filter any row */ rowkey = Bytes.toBytes("z"); Index: hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java (revision 1583858) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java (working copy) @@ -23,8 +23,12 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -67,10 +71,15 @@ private void testFiltersBeyondPageSize(final Filter f, final int pageSize) throws IOException { int count = 0; + List kvs = new ArrayList(); + byte [] rowkey = Bytes.toBytes("yyyyyyyyy"); + kvs.add(new KeyValue(rowkey, rowkey, Bytes.toBytes(0), + Bytes.toBytes(0))); for (int i = 0; i < (pageSize * 2); i++) { - boolean filterOut = f.filterRow(); + f.filterRow(kvs); + boolean filterOut = kvs.isEmpty(); - if(filterOut) { + if (filterOut) { break; } else { count++; Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (revision 1583858) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (working copy) @@ -3987,7 +3987,7 @@ 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()) + return filter != null && (filter.hasFilterRow()) && filter.filterRow(); } Index: hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java (revision 1583858) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java (working copy) @@ -331,9 +331,22 @@ // convert to List and call the new interface (this will call 0.96-style // #filterRowCells(List) which may delegate to legacy #filterRow(List) List cells = new ArrayList(kvs.size()); - cells.addAll(kvs); for (Filter filter : filters) { - filter.filterRowCells(cells); + // since single filter may modify the List of Cells, make a copy of the original + cells.clear(); + cells.addAll(kvs); + filter.filterRowCells(cells); + if (operator == Operator.MUST_PASS_ALL) { + if (cells.isEmpty()) { + kvs.clear(); + return; + } + } else if (operator == Operator.MUST_PASS_ONE) { + if (!cells.isEmpty()) { + return; + } + } + } // convert results into kvs Index: hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java (revision 1583858) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java (working copy) @@ -23,12 +23,14 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.Filter.ReturnCode; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; import java.io.IOException; import java.util.ArrayList; +import java.util.List; /** * Implementation of Filter interface that limits results to a specific page * size. It terminates scanning once the number of filter-passed rows is > @@ -45,6 +47,7 @@ public class PageFilter extends FilterBase { private long pageSize = Long.MAX_VALUE; private int rowsAccepted = 0; + private byte[] rowKey = null; /** * Constructor that takes a maximum page size. @@ -69,15 +72,25 @@ return this.rowsAccepted >= this.pageSize; } - public boolean filterRow() { - this.rowsAccepted++; - return this.rowsAccepted > this.pageSize; + @Override + public void filterRow(List kvs) throws IOException { + if (kvs.isEmpty()) { + return; + } + byte[] currKey = kvs.get(0).getKey(); + if (currKey != rowKey) { + rowKey = currKey; + this.rowsAccepted++; + } + if (this.rowsAccepted > this.pageSize) { + kvs.clear(); + } } public boolean hasFilterRow() { return true; } - + public static Filter createFilterFromArguments(ArrayList filterArguments) { Preconditions.checkArgument(filterArguments.size() == 1, "Expected 1 but got: %s", filterArguments.size()); Index: hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java (revision 1583858) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java (working copy) @@ -20,10 +20,12 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; +import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; @@ -85,6 +87,12 @@ } @Override + public void filterRow(List kvs) throws IOException { + final boolean wasEmptyRow = kvs.isEmpty(); + this.filter.filterRow(kvs); + changeFAR(!wasEmptyRow && kvs.isEmpty()); + } + @Override public boolean filterRow() throws IOException { boolean filterRow = this.filter.filterRow(); changeFAR(filterRow);