Index: src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (revision 7683) +++ src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (working copy) @@ -94,6 +94,7 @@ import org.apache.hadoop.hbase.client.coprocessor.ExecResult; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FilterWrapper; import org.apache.hadoop.hbase.filter.IncompatibleFilterException; import org.apache.hadoop.hbase.filter.WritableByteArrayComparable; import org.apache.hadoop.hbase.io.HeapSize; @@ -3225,7 +3226,7 @@ // Package local for testability KeyValueHeap storeHeap = null; private final byte [] stopRow; - private Filter filter; + private FilterWrapper filter; private List results = new ArrayList(); private int batch; private int isScan; @@ -3237,8 +3238,13 @@ } RegionScannerImpl(Scan scan, List additionalScanners) throws IOException { //DebugPrint.println("HRegionScanner."); - - this.filter = scan.getFilter(); + + if(scan.hasFilter()){ + this.filter = new FilterWrapper(scan.getFilter()); + }else { + this.filter = null; + } + this.batch = scan.getBatch(); if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) { this.stopRow = null; @@ -3341,9 +3347,6 @@ if (filter != null && filter.hasFilterRow()) { filter.filterRow(results); } - if (filter != null && filter.filterRow()) { - results.clear(); - } return false; } else if (filterRowKey(currentRow)) { @@ -3355,7 +3358,7 @@ if (limit > 0 && results.size() == limit) { if (this.filter != null && filter.hasFilterRow()) { throw new IncompatibleFilterException( - "Filter with filterRow(List) incompatible with scan with limit!"); + "Filter with filterRow(List) or filterRow() incompatible with scan with limit!"); } return true; // we are expecting more yes, but also limited to how many we can return. } @@ -3370,7 +3373,7 @@ filter.filterRow(results); } - if (results.isEmpty() || filterRow()) { + if (results.isEmpty()) { // this seems like a redundant step - we already consumed the row // there're no left overs. // the reasons for calling this method are: Index: src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java (revision 7683) +++ src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java (working copy) @@ -84,6 +84,10 @@ public boolean filterRow() { return filterRow; } + + public boolean hasFilterRow() { + return true; + } public void write(DataOutput out) throws IOException { out.writeUTF(this.filter.getClass().getName()); Index: src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java (revision 7683) +++ src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java (working copy) @@ -73,6 +73,10 @@ this.rowsAccepted++; return this.rowsAccepted > this.pageSize; } + + public boolean hasFilterRow() { + return true; + } public static Filter createFilterFromArguments(ArrayList filterArguments) { Preconditions.checkArgument(filterArguments.size() == 1, Index: src/main/java/org/apache/hadoop/hbase/filter/Filter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/Filter.java (revision 7683) +++ src/main/java/org/apache/hadoop/hbase/filter/Filter.java (working copy) @@ -139,7 +139,7 @@ public void filterRow(List kvs); /** - * @return True if this filter actively uses filterRow(List). + * @return True if this filter actively uses filterRow(List) and filterRow(). * Primarily used to check for conflicts with scans(such as scans * that do not read a full row at a time) */ Index: src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java (revision 7683) +++ src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java (working copy) @@ -85,6 +85,10 @@ changeFAR(filterRow); return filterRow; } + + public boolean hasFilterRow() { + return true; + } public void write(DataOutput out) throws IOException { out.writeUTF(this.filter.getClass().getName()); Index: src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (revision 7683) +++ src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (working copy) @@ -193,6 +193,12 @@ } } + + public boolean hasFilterRow(){ + //return true since either filterRow() or filterRow(List kvs) is defined + return true; + } + public boolean filterRow() { // If column was found, return false if it was matched, true if it was not // If column not found, return true if we filter if missing, false if not Index: src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java (revision 7683) +++ src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java (working copy) @@ -86,6 +86,10 @@ return filterOutRow; } + public boolean hasFilterRow() { + return true; + } + @Override public boolean filterRowKey(byte[] buffer, int offset, int length) { if (chance < 0) { Index: src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java (revision 0) @@ -0,0 +1,85 @@ +package org.apache.hadoop.hbase.filter; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue; + +/** + * This is a Filter wrapper class which is used in the server side. + * Some filter related hooks can be defined in this wrapper. + * The only way to create a FilterWrapper instance is passing a client side Filter instance. + * + */ +public class FilterWrapper implements Filter { + Filter filter = null; + + public FilterWrapper(Filter filter) { + if(null == filter) { + //ensure the filter instance is not null + throw new NullPointerException("Cannot create FilterWrapper with null"); + } + this.filter = filter; + } + + @Override + public void write(DataOutput out) throws IOException{ + this.filter.write(out); + } + + @Override + public void readFields(DataInput in) throws IOException{ + this.filter.readFields(in); + } + + @Override + public void reset(){ + this.filter.reset(); + } + + @Override + public boolean filterAllRemaining() { + return this.filter.filterAllRemaining(); + } + + @Override + public boolean filterRow(){ + return this.filter.filterRow(); + } + + @Override + public KeyValue getNextKeyHint(KeyValue currentKV){ + return this.filter.getNextKeyHint(currentKV); + } + + @Override + public boolean filterRowKey(byte[] buffer, int offset, int length){ + return this.filter.filterRowKey(buffer, offset, length); + } + + @Override + public ReturnCode filterKeyValue(KeyValue v){ + return this.filter.filterKeyValue(v); + } + + @Override + public KeyValue transform(KeyValue v){ + return this.filter.transform(v); + } + + + @Override + public boolean hasFilterRow(){ + return this.filter.hasFilterRow(); + } + + //Wrap the following function calls + @Override + public void filterRow(List kvs){ + this.filter.filterRow(kvs); + if(this.filter.filterRow()) kvs.clear(); + } + +}