Index: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (revision 1571830) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (working copy) @@ -516,6 +516,10 @@ syncBlockingFilter.set(true); return false; } + @Override + public ReturnCode filterKeyValue(Cell ignored) throws IOException { + return ReturnCode.INCLUDE; + } public static Filter parseFrom(final byte [] pbBytes) throws DeserializationException{ return new BlockingFilter(); Index: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (revision 1571830) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (working copy) @@ -96,6 +96,7 @@ import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.ColumnCountGetFilter; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.Filter.ReturnCode; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.filter.FilterList; @@ -2639,6 +2640,10 @@ scan.setLoadColumnFamiliesOnDemand(true); Filter bogusFilter = new FilterBase() { @Override + public ReturnCode filterKeyValue(Cell ignored) throws IOException { + return ReturnCode.INCLUDE; + } + @Override public boolean isFamilyEssential(byte[] name) { return Bytes.equals(name, cf_first); } 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 1571830) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java (working copy) @@ -419,7 +419,14 @@ Filter filterNoHint = new FilterBase() { @Override - public byte [] toByteArray() {return null;} + public byte [] toByteArray() { + return null; + } + + @Override + public ReturnCode filterKeyValue(Cell ignored) throws IOException { + return ReturnCode.INCLUDE; + } }; Filter filterMinHint = new FilterBase() { Index: hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java (revision 1571830) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java (working copy) @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.Filter.ReturnCode; import org.apache.hadoop.hbase.filter.FilterList.Operator; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; @@ -598,6 +599,11 @@ // always filter out rows return true; } + + @Override + public ReturnCode filterKeyValue(Cell ignored) throws IOException { + return ReturnCode.INCLUDE; + } } /** @@ -1985,6 +1991,11 @@ public byte [] toByteArray() {return null;} @Override + public ReturnCode filterKeyValue(Cell ignored) throws IOException { + return ReturnCode.INCLUDE; + } + + @Override public boolean filterRow() throws IOException { ipcHandlerThread = Thread.currentThread(); try { Index: hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (revision 1571830) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (working copy) @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter; +import java.io.IOException; import java.util.ArrayList; import org.apache.hadoop.classification.InterfaceAudience; @@ -27,6 +28,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.filter.Filter.ReturnCode; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; import com.google.common.base.Preconditions; @@ -57,6 +59,11 @@ return v.createKeyOnly(this.lenAsVal); } + @Override + public ReturnCode filterKeyValue(Cell ignored) throws IOException { + return ReturnCode.INCLUDE; + } + public static Filter createFilterFromArguments(ArrayList filterArguments) { Preconditions.checkArgument((filterArguments.size() == 0 || filterArguments.size() == 1), "Expected: 0 or 1 but got: %s", filterArguments.size()); Index: hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java (revision 1571830) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java (working copy) @@ -73,17 +73,6 @@ } /** - * Filters that dont filter by key value can inherit this implementation that - * includes all Cells. - * - * @inheritDoc - */ - @Override - public ReturnCode filterKeyValue(Cell ignored) throws IOException { - return ReturnCode.INCLUDE; - } - - /** * By default no transformation takes place * * @inheritDoc 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 1571830) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java (working copy) @@ -22,9 +22,12 @@ import com.google.protobuf.InvalidProtocolBufferException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.Cell; 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; /** * Implementation of Filter interface that limits results to a specific page @@ -57,6 +60,11 @@ return pageSize; } + @Override + public ReturnCode filterKeyValue(Cell ignored) throws IOException { + return ReturnCode.INCLUDE; + } + public boolean filterAllRemaining() { return this.rowsAccepted >= this.pageSize; } Index: hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java (revision 1571830) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java (working copy) @@ -98,6 +98,10 @@ * code is described below. This allows filters to filter only certain number of columns, then * terminate without matching ever column. * + * If filterRowKey returns true, filterKeyValue needs to be consistent with it. + * + * filterKeyValue can assume that filterRowKey has already been called for the row. + * * If your filter returns ReturnCode.NEXT_ROW, it should return * ReturnCode.NEXT_ROW until {@link #reset()} is called just in case the caller calls * for the next row.