Index: hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java (revision 1582242) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java (working copy) @@ -20,18 +20,15 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; -import java.util.ArrayList; 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.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; -import org.apache.zookeeper.KeeperException.UnimplementedException; import com.google.protobuf.InvalidProtocolBufferException; @@ -154,15 +151,7 @@ @Override public void filterRowCells(List kvs) throws IOException { - //To fix HBASE-6429, - //Filter with filterRow() returning true is incompatible with scan with limit - //1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented. - //2. filterRow() is merged with filterRow(kvs), - //so that to make all those row related filtering stuff in the same function. this.filter.filterRowCells(kvs); - if (!kvs.isEmpty() && this.filter.filterRow()) { - kvs.clear(); - } } /** Index: hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (revision 1582242) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (working copy) @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.List; import com.google.protobuf.HBaseZeroCopyByteString; import org.apache.commons.logging.Log; @@ -239,6 +240,12 @@ return filterIfMissing; } + @Override + public void filterRowCells(List ignored) throws IOException { + // Just do nothing. This is implemented here so that we can avoid the operations of new KV list + // creation etc in super class method ie.FilterBase#filterRowCells(List ignored). + } + /** * Set whether entire row should be filtered if column is not found. *

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 1582242) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (working copy) @@ -3977,20 +3977,10 @@ } } - /** - * 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(); + return filter != null && filter.filterRow(); } - + private boolean filterRowKey(byte[] row, int offset, short length) throws IOException { return filter != null && filter.filterRowKey(row, offset, length);