Index: src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java =================================================================== --- src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java (revision 649252) +++ src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java (working copy) @@ -134,7 +134,7 @@ } // Filter whole row by row key? - filtered = dataFilter != null? dataFilter.filter(chosenRow) : false; + filtered = dataFilter != null? dataFilter.filterRowKey(chosenRow) : false; // Store the key and results for each sub-scanner. Merge them as // appropriate. @@ -187,7 +187,7 @@ if (dataFilter != null) { // Filter whole row by column data? filtered = - dataFilter.filter(chosenRow, e.getKey(), e.getValue()); + dataFilter.filterColumn(chosenRow, e.getKey(), e.getValue()); if (filtered) { results.clear(); break; Index: src/java/org/apache/hadoop/hbase/regionserver/HRegion.java =================================================================== --- src/java/org/apache/hadoop/hbase/regionserver/HRegion.java (revision 649252) +++ src/java/org/apache/hadoop/hbase/regionserver/HRegion.java (working copy) @@ -1803,7 +1803,7 @@ moreToFollow = false; } - filtered = filter == null ? false : filter.filterNotNull(results); + filtered = filter == null ? false : filter.filterRow(results); if (filtered && moreToFollow) { results.clear(); Index: src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java =================================================================== --- src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java (revision 649252) +++ src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java (working copy) @@ -97,7 +97,7 @@ } /** {@inheritDoc} */ - public boolean filter(final Text rowKey) { + public boolean filterRowKey(final Text rowKey) { if (rowKey == null) { if (this.stopRowKey == null) { return true; @@ -118,10 +118,10 @@ * Because StopRowFilter does not examine column information, this method * defaults to calling the rowKey-only version of filter. */ - public boolean filter(@SuppressWarnings("unused") final Text rowKey, + public boolean filterColumn(@SuppressWarnings("unused") final Text rowKey, @SuppressWarnings("unused") final Text colKey, @SuppressWarnings("unused") final byte[] data) { - return filter(rowKey); + return filterRowKey(rowKey); } /** {@inheritDoc} @@ -131,7 +131,7 @@ * * @param columns */ - public boolean filterNotNull(@SuppressWarnings("unused") + public boolean filterRow(@SuppressWarnings("unused") final SortedMap columns) { return filterAllRemaining(); } Index: src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java =================================================================== --- src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java (revision 649252) +++ src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java (working copy) @@ -93,8 +93,8 @@ } /** {@inheritDoc} */ - public boolean filter(final Text rowKey) { - changeFAR(this.filter.filter(rowKey)); + public boolean filterRowKey(final Text rowKey) { + changeFAR(this.filter.filterRowKey(rowKey)); boolean result = filterAllRemaining(); if (LOG.isDebugEnabled()) { LOG.debug("Filter on rowKey:" + rowKey + ". Result = " + result); @@ -103,9 +103,9 @@ } /** {@inheritDoc} */ - public boolean filter(final Text rowKey, final Text colKey, + public boolean filterColumn(final Text rowKey, final Text colKey, final byte[] data) { - changeFAR(this.filter.filter(rowKey, colKey, data)); + changeFAR(this.filter.filterColumn(rowKey, colKey, data)); boolean result = filterAllRemaining(); if (LOG.isDebugEnabled()) { LOG.debug("Filter on rowKey:" + rowKey + ", colKey: " + colKey + @@ -115,8 +115,8 @@ } /** {@inheritDoc} */ - public boolean filterNotNull(final SortedMap columns) { - changeFAR(this.filter.filterNotNull(columns)); + public boolean filterRow(final SortedMap columns) { + changeFAR(this.filter.filterRow(columns)); boolean result = filterAllRemaining(); if (LOG.isDebugEnabled()) { LOG.debug("FilterNotNull on cols:" + columns + ". Result = " + Index: src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java =================================================================== --- src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java (revision 649252) +++ src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java (working copy) @@ -43,7 +43,7 @@ /** {@inheritDoc} */ @Override - public boolean filter(final Text rowKey) { + public boolean filterRowKey(final Text rowKey) { if (rowKey == null) { if (this.stopRowKey == null) { return true; Index: src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java =================================================================== --- src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java (revision 649252) +++ src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java (working copy) @@ -145,7 +145,7 @@ * * {@inheritDoc} */ - public boolean filter(final Text rowKey) { + public boolean filterRowKey(final Text rowKey) { if (filtersByRowKey() && rowKey != null) { boolean result = !getRowKeyPattern().matcher(rowKey.toString()).matches(); if (LOG.isDebugEnabled()) { @@ -160,9 +160,9 @@ * * {@inheritDoc} */ - public boolean filter(final Text rowKey, final Text colKey, + public boolean filterColumn(final Text rowKey, final Text colKey, final byte[] data) { - if (filter(rowKey)) { + if (filterRowKey(rowKey)) { return true; } if (filtersByColumnValue()) { @@ -196,7 +196,7 @@ * * {@inheritDoc} */ - public boolean filterNotNull(final SortedMap columns) { + public boolean filterRow(final SortedMap columns) { for (Entry col : columns.entrySet()) { if (nullColumns.contains(col.getKey()) && !HLogEdit.isDeleted(col.getValue())) { Index: src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java =================================================================== --- src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java (revision 649252) +++ src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java (working copy) @@ -118,7 +118,7 @@ * * {@inheritDoc} */ - public boolean filter(@SuppressWarnings("unused") final Text rowKey) { + public boolean filterRowKey(@SuppressWarnings("unused") final Text rowKey) { return filterAllRemaining(); } @@ -126,7 +126,7 @@ * * {@inheritDoc} */ - public boolean filter(@SuppressWarnings("unused") final Text rowKey, + public boolean filterColumn(@SuppressWarnings("unused") final Text rowKey, @SuppressWarnings("unused") final Text colKey, @SuppressWarnings("unused") final byte[] data) { return filterAllRemaining(); @@ -136,7 +136,7 @@ * * {@inheritDoc} */ - public boolean filterNotNull(@SuppressWarnings("unused") + public boolean filterRow(@SuppressWarnings("unused") final SortedMap columns) { return filterAllRemaining(); } Index: src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java =================================================================== --- src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java (revision 649252) +++ src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java (working copy) @@ -74,41 +74,31 @@ boolean filterAllRemaining(); /** - * Filters on just a row key. + * Filters on just a row key. This is the first chance to stop a row. * * @param rowKey * @return true if given row key is filtered and row should not be processed. */ - boolean filter(final Text rowKey); + boolean filterRowKey(final Text rowKey); /** - * Filters on row key and/or a column key. + * Filters on row key, column name, and column value. This will take individual columns out of a row, + * but the rest of the row will still get through. * - * @param rowKey - * row key to filter on. May be null for no filtering of row key. - * @param colKey - * column whose data will be filtered - * @param data - * column value + * @param rowKey row key to filter on. + * @param colunmName column name to filter on + * @param columnValue column value to filter on * @return true if row filtered and should not be processed. */ - boolean filter(final Text rowKey, final Text colKey, final byte[] data); + boolean filterColumn(final Text rowKey, final Text colunmName, final byte[] columnValue); /** - * Filters a row if: - * 1) The given row (@param columns) has a columnKey expected to be null AND - * the value associated with that columnKey is non-null. - * 2) The filter has a criterion for a particular columnKey, but that - * columnKey is not in the given row (@param columns). + * Filter on the fully assembled row. This is the last chance to stop a row. * - * Note that filterNotNull does not care whether the values associated with a - * columnKey match. Also note that a "null value" associated with a columnKey - * is expressed as HConstants.DELETE_BYTES. - * * @param columns - * @return true if null/non-null criteria not met. + * @return true if row filtered and should not be processed. */ - boolean filterNotNull(final SortedMap columns); + boolean filterRow(final SortedMap columns); /** * Validates that this filter applies only to a subset of the given columns. Index: src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java =================================================================== --- src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java (revision 649252) +++ src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java (working copy) @@ -28,6 +28,9 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.io.ObjectWritable; import org.apache.hadoop.io.Text; /** @@ -157,13 +160,13 @@ } /** {@inheritDoc} */ - public boolean filter(final Text rowKey) { + public boolean filterRowKey(final Text rowKey) { boolean resultFound = false; boolean result = operator == Operator.MUST_PASS_ONE; for (RowFilterInterface filter : filters) { if (!resultFound) { if (operator == Operator.MUST_PASS_ALL) { - if (filter.filterAllRemaining() || filter.filter(rowKey)) { + if (filter.filterAllRemaining() || filter.filterRowKey(rowKey)) { if (LOG.isDebugEnabled()) { LOG.debug("op.MPALL filter(Text) will return true due" + " to subfilter of type " + filter.getClass().getSimpleName()); @@ -172,7 +175,7 @@ resultFound = true; } } else if (operator == Operator.MUST_PASS_ONE) { - if (!filter.filterAllRemaining() && !filter.filter(rowKey)) { + if (!filter.filterAllRemaining() && !filter.filterRowKey(rowKey)) { if (LOG.isDebugEnabled()) { LOG.debug("op.MPONE filter(Text) will return false due" + " to subfilter of type " + filter.getClass().getSimpleName()); @@ -182,7 +185,7 @@ } } } else if (filter.processAlways()) { - filter.filter(rowKey); + filter.filterRowKey(rowKey); } } if (LOG.isDebugEnabled()) { @@ -192,7 +195,7 @@ } /** {@inheritDoc} */ - public boolean filter(final Text rowKey, final Text colKey, + public boolean filterColumn(final Text rowKey, final Text colKey, final byte[] data) { boolean resultFound = false; boolean result = operator == Operator.MUST_PASS_ONE; @@ -200,7 +203,7 @@ if (!resultFound) { if (operator == Operator.MUST_PASS_ALL) { if (filter.filterAllRemaining() || - filter.filter(rowKey, colKey, data)) { + filter.filterColumn(rowKey, colKey, data)) { if (LOG.isDebugEnabled()) { LOG.debug("op.MPALL filter(Text, Text, byte[]) will" + " return true due to subfilter of type " + @@ -211,7 +214,7 @@ } } else if (operator == Operator.MUST_PASS_ONE) { if (!filter.filterAllRemaining() && - !filter.filter(rowKey, colKey, data)) { + !filter.filterColumn(rowKey, colKey, data)) { if (LOG.isDebugEnabled()) { LOG.debug("op.MPONE filter(Text, Text, byte[]) will" + " return false due to subfilter of type " + @@ -222,7 +225,7 @@ } } } else if (filter.processAlways()) { - filter.filter(rowKey, colKey, data); + filter.filterColumn(rowKey, colKey, data); } } if (LOG.isDebugEnabled()) { @@ -232,13 +235,13 @@ } /** {@inheritDoc} */ - public boolean filterNotNull(final SortedMap columns) { + public boolean filterRow(final SortedMap columns) { boolean resultFound = false; boolean result = operator == Operator.MUST_PASS_ONE; for (RowFilterInterface filter : filters) { if (!resultFound) { if (operator == Operator.MUST_PASS_ALL) { - if (filter.filterAllRemaining() || filter.filterNotNull(columns)) { + if (filter.filterAllRemaining() || filter.filterRow(columns)) { if (LOG.isDebugEnabled()) { LOG.debug("op.MPALL filterNotNull will return true due" + " to subfilter of type " + filter.getClass().getSimpleName()); @@ -247,7 +250,7 @@ resultFound = true; } } else if (operator == Operator.MUST_PASS_ONE) { - if (!filter.filterAllRemaining() && !filter.filterNotNull(columns)) { + if (!filter.filterAllRemaining() && !filter.filterRow(columns)) { if (LOG.isDebugEnabled()) { LOG.debug("op.MPONE filterNotNull will return false due" + " to subfilter of type " + filter.getClass().getSimpleName()); @@ -257,7 +260,7 @@ } } } else if (filter.processAlways()) { - filter.filterNotNull(columns); + filter.filterRow(columns); } } if (LOG.isDebugEnabled()) { @@ -268,46 +271,32 @@ /** {@inheritDoc} */ public void readFields(final DataInput in) throws IOException { + Configuration conf = new HBaseConfiguration(); byte opByte = in.readByte(); operator = Operator.values()[opByte]; int size = in.readInt(); if (size > 0) { filters = new HashSet(); - try { - for (int i = 0; i < size; i++) { - String className = in.readUTF(); - Class clazz = Class.forName(className); - RowFilterInterface filter; - filter = (RowFilterInterface) clazz.newInstance(); - filter.readFields(in); - filters.add(filter); - if (LOG.isDebugEnabled()) { - LOG.debug("Successfully read in subfilter of type " + - filter.getClass().getSimpleName()); - } - } - } catch (InstantiationException e) { - throw new RuntimeException("Failed to deserialize RowFilterInterface.", - e); - } catch (IllegalAccessException e) { - throw new RuntimeException("Failed to deserialize RowFilterInterface.", - e); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Failed to deserialize RowFilterInterface.", - e); + for (int i = 0; i < size; i++) { + RowFilterInterface filter = (RowFilterInterface) ObjectWritable + .readObject(in, conf); + filters.add(filter); + if (LOG.isDebugEnabled()) { + LOG.debug("Successfully read in subfilter of type " + + filter.getClass().getSimpleName()); + } } } - } /** {@inheritDoc} */ public void write(final DataOutput out) throws IOException { + Configuration conf = new HBaseConfiguration(); out.writeByte(operator.ordinal()); out.writeInt(filters.size()); for (RowFilterInterface filter : filters) { - out.writeUTF(filter.getClass().getName()); - filter.write(out); + ObjectWritable.writeObject(out, filter, RowFilterInterface.class, conf); } } - + }