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 1591992) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (working copy) @@ -4013,7 +4013,7 @@ ret = filter.filterRowCellsWithRet(results); } - if ((isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE) || filterRow()) { + if ((isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE)) { results.clear(); boolean moreRows = nextRow(currentRow, offset, length); if (!moreRows) return false; @@ -4067,20 +4067,6 @@ } } - /** - * 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(); - } - private boolean filterRowKey(byte[] row, int offset, short length) throws IOException { return filter != null && filter.filterRowKey(row, offset, length); 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 1591992) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java (working copy) @@ -80,6 +80,8 @@ * Transformations are composed in the order specified by {@link #filters}. */ private Cell transformedKV = null; + + private Boolean hasFilterRow = null; /** * Constructor that takes a set of {@link Filter}s. The default operator @@ -342,15 +344,19 @@ kvs.add(KeyValueUtil.ensureKeyValue(c)); } } - + @Override public boolean hasFilterRow() { - for (Filter filter : filters) { - if(filter.hasFilterRow()) { - return true; + if (hasFilterRow == null) { + for (Filter filter : filters) { + if (FilterUtil.hasFilterRow(filter)) { + this.hasFilterRow = true; + return true; + } } + this.hasFilterRow = false; } - return false; + return hasFilterRow; } @Override Index: hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterUtil.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterUtil.java (revision 0) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterUtil.java (revision 0) @@ -0,0 +1,83 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.filter; + +import java.lang.reflect.Method; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * This class provides utility methods for Filters + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +final public class FilterUtil { + static final Log LOG = LogFactory.getLog(FilterUtil.class); + /* + * 'Implemented' means the method is implemented and not inherited from FilterBase + * @param filter the filter object + * @param name method name + * @param classes classes of the parameters + * @return true if named method is implemented + */ + public static boolean isFilterMethodImplemented(Filter filter, String name, Class [] classes) + { + Class baseCls = FilterBase.class.getClass(); + Class filterCls = Filter.class.getClass(); + try + { + Class clazz = filter.getClass(); + Method m = clazz.getMethod(name, classes); + Class declaringClass = m.getDeclaringClass(); + Class superCls = declaringClass.getSuperclass(); + if (declaringClass.equals(clazz) && superCls.equals(filterCls)) { + // filter class directly overrides Filter + return filter.hasFilterRow(); + } + return !declaringClass.equals(baseCls); + } catch (SecurityException e) { + LOG.error("Unable to determine whether " + name + " is implemented", e); + } catch (NoSuchMethodException e) { + LOG.error(name + " doesn't exist", e); + } + + return false; + } + + /* + * This is a utility method that checks the type of Filter + * and calls corresponding method + * @param filter the filter instance + * @return whether the filter implements filterRow() method + */ + public static boolean hasFilterRow(Filter filter) { + if (filter instanceof FilterList) { + // FilterList#hasFilterRow() is implemented + if (filter.hasFilterRow()) { + return true; + } + } else if (FilterUtil.isFilterMethodImplemented(filter, "filterRow", new Class[]{})) { + return true; + } + return false; + } +} 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 1591992) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java (working copy) @@ -20,11 +20,12 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; -import java.util.ArrayList; +import java.lang.reflect.Method; import java.util.List; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; 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; @@ -43,7 +44,10 @@ */ @InterfaceAudience.Private final public class FilterWrapper extends Filter { + static final Log LOG = LogFactory.getLog(FilterWrapper.class); Filter filter = null; + private boolean hasFilterRow; + private boolean hasFilterRowCells; public FilterWrapper( Filter filter ) { if (null == filter) { @@ -51,6 +55,10 @@ throw new NullPointerException("Cannot create FilterWrapper with null Filter"); } this.filter = filter; + this.hasFilterRow = FilterUtil.isFilterMethodImplemented(filter, "filterRow", new Class []{}); + this.hasFilterRowCells = FilterUtil.isFilterMethodImplemented(filter, "filterRowCells", + new Class [] { List.class.getClass() }) || FilterUtil.isFilterMethodImplemented( + filter, "filterRow", new Class [] { List.class.getClass() }); } /** @@ -148,7 +156,7 @@ @Override public boolean hasFilterRow() { - return this.filter.hasFilterRow(); + return this.hasFilterRow || this.hasFilterRowCells; } @Override @@ -168,7 +176,7 @@ //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()) { + if (!kvs.isEmpty() && this.hasFilterRow) { if (this.filter.filterRow()) { kvs.clear(); return FilterRowRetCode.EXCLUDE; Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java (revision 1591992) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java (working copy) @@ -25,6 +25,9 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterUtil; +import org.apache.hadoop.hbase.filter.FilterWrapper; import org.apache.hadoop.hbase.filter.IncompatibleFilterException; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.util.Bytes; @@ -376,10 +379,11 @@ * @param batch the maximum number of values */ public void setBatch(int batch) { - if (this.hasFilter() && this.filter.hasFilterRow()) { - throw new IncompatibleFilterException( - "Cannot set batch on a scan using a filter" + - " that returns true for filter.hasFilterRow"); + if (this.hasFilter()) { + if (FilterUtil.hasFilterRow(this.filter)) { + throw new IncompatibleFilterException( + "Cannot set batch on a scan using a filter that overrides filterRow()"); + } } this.batch = batch; }