You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2009/07/01 07:32:55 UTC

svn commit: r790043 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/filter/ src/java/org/apache/hadoop/hbase/io/ src/test/org/apache/hadoop/hbase/filter/

Author: stack
Date: Wed Jul  1 05:32:54 2009
New Revision: 790043

URL: http://svn.apache.org/viewvc?rev=790043&view=rev
Log:
HBASE-1582 Translate ColumnValueFilter and RowFilterSet to the new Filter interface

Added:
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java
      - copied, changed from r789818, hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestInclusiveStopRowFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestValueFilter.java
      - copied, changed from r789818, hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java
Removed:
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowPrefixFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestInclusiveStopRowFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestPrefixRowFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRegExpRowFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterOnMultipleFamilies.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterSet.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowPrefixFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java
Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ValueFilter.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/package-info.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestFilterSet.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=790043&r1=790042&r2=790043&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Wed Jul  1 05:32:54 2009
@@ -426,6 +426,8 @@
    HBASE-1385  Revamp TableInputFormat, needs updating to match hadoop 0.20.x
                AND remove bit where we can make < maps than regions
                (Lars George via Stack)
+   HBASE-1582  Translate ColumnValueFilter and RowFilterSet to the new
+               Filter interface (Clint Morgan and Stack)
 
   OPTIMIZATIONS
    HBASE-1412  Change values for delete column and column family in KeyValue

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java?rev=790043&r1=790042&r2=790043&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java Wed Jul  1 05:32:54 2009
@@ -41,7 +41,7 @@
  * lexicographic compare. If this is not sufficient (eg you want to deserialize
  * a long and then compare it to a fixed long value), then you can pass in your
  * own comparator instead.
- * @deprecated Use filters that are rooted on @{link Filter} instead
+ * @deprecated Use filters that are rooted on @{link Filter} instead.
  */
 public class ColumnValueFilter implements RowFilterInterface {
   /** Comparison operators. */

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java?rev=790043&r1=790042&r2=790043&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java Wed Jul  1 05:32:54 2009
@@ -56,7 +56,8 @@
       return false;
     }
     // if stopRowKey is <= buffer, then true, filter row.
-    return Bytes.compareTo(stopRowKey, 0, stopRowKey.length, buffer, offset, length) < 0;
+    return Bytes.compareTo(stopRowKey, 0, stopRowKey.length,
+      buffer, offset, length) < 0;
   }
 
   public boolean filterAllRemaining() {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java?rev=790043&r1=790042&r2=790043&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java Wed Jul  1 05:32:54 2009
@@ -43,9 +43,8 @@
  * 
  * Note that column value filtering in this interface has been replaced by
  * {@link ColumnValueFilter}.
- * @deprecated This interface doesn't really work well in new KeyValue world.
- * Needs to be refactored/removed.  Marking it as deprecated till it gets
- * cleaned up.  Its also inefficient as written.
+ * @deprecated This interface doesn't work well in new KeyValue world.
+ * Use filters based on new {@link Filter} instead.
  */
 public class RegExpRowFilter implements RowFilterInterface {
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ValueFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ValueFilter.java?rev=790043&r1=790042&r2=790043&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ValueFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ValueFilter.java Wed Jul  1 05:32:54 2009
@@ -35,7 +35,9 @@
  * This filter is used to filter based on the value of a given column. It takes
  * an operator (equal, greater, not equal, etc) and either a byte [] value or a
  * byte [] comparator. If we have a byte [] value then we just do a
- * lexicographic compare. If this is not sufficient (eg you want to deserialize
+ * lexicographic compare. For example, if passed value is 'b' and cell has 'a'
+ * and the compare operator is LESS, then we will filter out this cell (return
+ * true).  If this is not sufficient (eg you want to deserialize
  * a long and then compare it to a fixed long value), then you can pass in your
  * own comparator instead.
  * */
@@ -58,12 +60,16 @@
     GREATER;
   }
 
-  private byte[] columnName;
+  private byte [] columnFamily;
+  private byte [] columnQualifier; 
   private CompareOp compareOp;
-  private byte[] value;
+  private byte [] value;
   private WritableByteArrayComparable comparator;
   private boolean filterIfColumnMissing;
 
+  private boolean filterThisRow = false;
+  private boolean foundColValue = false;
+
   ValueFilter() {
     // for Writable
   }
@@ -71,27 +77,31 @@
   /**
    * Constructor.
    * 
-   * @param columnName name of column
+   * @param family name of column family
+   * @param qualifier name of column qualifier
    * @param compareOp operator
    * @param value value to compare column values against
    */
-  public ValueFilter(final byte[] columnName, final CompareOp compareOp,
-      final byte[] value) {
-    this(columnName, compareOp, value, true);
+  public ValueFilter(final byte [] family, final byte [] qualifier,
+      final CompareOp compareOp, final byte[] value) {
+    this(family, qualifier, compareOp, value, true);
   }
 
   /**
    * Constructor.
    * 
-   * @param columnName name of column
+   * @param family name of column family
+   * @param qualifier name of column qualifier
    * @param compareOp operator
    * @param value value to compare column values against
    * @param filterIfColumnMissing if true then we will filter rows that don't
    * have the column.
    */
-  public ValueFilter(final byte[] columnName, final CompareOp compareOp,
+  public ValueFilter(final byte [] family, final byte [] qualifier,
+      final CompareOp compareOp,
       final byte[] value, boolean filterIfColumnMissing) {
-    this.columnName = columnName;
+    this.columnFamily = family;
+    this.columnQualifier = qualifier;
     this.compareOp = compareOp;
     this.value = value;
     this.filterIfColumnMissing = filterIfColumnMissing;
@@ -100,28 +110,33 @@
   /**
    * Constructor.
    * 
-   * @param columnName name of column
+   * @param family name of column family
+   * @param qualifier name of column qualifier
    * @param compareOp operator
    * @param comparator Comparator to use.
    */
-  public ValueFilter(final byte[] columnName, final CompareOp compareOp,
+  public ValueFilter(final byte [] family, final byte [] qualifier,
+      final CompareOp compareOp,
       final WritableByteArrayComparable comparator) {
-    this(columnName, compareOp, comparator, true);
+    this(family, qualifier, compareOp, comparator, true);
   }
 
   /**
    * Constructor.
    * 
-   * @param columnName name of column
+   * @param family name of column family
+   * @param qualifier name of column qualifier
    * @param compareOp operator
    * @param comparator Comparator to use.
    * @param filterIfColumnMissing if true then we will filter rows that don't
    * have the column.
    */
-  public ValueFilter(final byte[] columnName, final CompareOp compareOp,
+  public ValueFilter(final byte [] family, final byte [] qualifier,
+      final CompareOp compareOp,
       final WritableByteArrayComparable comparator,
       boolean filterIfColumnMissing) {
-    this.columnName = columnName;
+    this.columnFamily = family;
+    this.columnQualifier = qualifier;
     this.compareOp = compareOp;
     this.comparator = comparator;
     this.filterIfColumnMissing = filterIfColumnMissing;
@@ -131,22 +146,15 @@
     return false;
   }
 
-  private boolean filterThisRow = false;
-  private boolean foundColValue = false;
-
   public ReturnCode filterKeyValue(KeyValue keyValue) {
-    if (Bytes.compareTo(keyValue.getColumn(), this.columnName) != 0) {
+    if (!keyValue.matchingColumn(this.columnFamily, this.columnQualifier)) {
       return ReturnCode.INCLUDE;
     }
-    LOG.info("Found column [" + Bytes.toString(columnName) + "] in row ["
-        + Bytes.toString(keyValue.getRow()) + "]");
-    foundColValue = true;
-
-    boolean filtered = filterColumnValue(keyValue.getBuffer(), keyValue
-        .getValueOffset(), keyValue.getValueLength());
+    this.foundColValue = true;
+    boolean filtered = filterColumnValue(keyValue.getBuffer(),
+      keyValue.getValueOffset(), keyValue.getValueLength());
     if (filtered) {
-      LOG.info("filtered it");
-      filterThisRow = true;
+      this.filterThisRow = true;
       return ReturnCode.NEXT_ROW;
     }
     return ReturnCode.INCLUDE;
@@ -189,7 +197,6 @@
     boolean result = filterThisRow || (filterIfColumnMissing && !foundColValue);
     filterThisRow = false;
     foundColValue = false;
-    LOG.info("Deciding " + (result ? "" : " not ") + "to filter");
     return result;
   }
 
@@ -203,7 +210,8 @@
       value = new byte[valueLen];
       in.readFully(value);
     }
-    columnName = Bytes.readByteArray(in);
+    this.columnFamily = Bytes.readByteArray(in);
+    this.columnQualifier = Bytes.readByteArray(in);
     compareOp = CompareOp.valueOf(in.readUTF());
     comparator = (WritableByteArrayComparable) ObjectWritable.readObject(in,
         new HBaseConfiguration());
@@ -217,7 +225,8 @@
       out.writeInt(value.length);
       out.write(value);
     }
-    Bytes.writeByteArray(out, columnName);
+    Bytes.writeByteArray(out, this.columnFamily);
+    Bytes.writeByteArray(out, this.columnQualifier);
     out.writeUTF(compareOp.name());
     ObjectWritable.writeObject(out, comparator,
         WritableByteArrayComparable.class, new HBaseConfiguration());

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java?rev=790043&r1=790042&r2=790043&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java Wed Jul  1 05:32:54 2009
@@ -22,7 +22,6 @@
 import org.apache.hadoop.io.Writable;
 
 /** Interface for both Comparable<byte []> and Writable. */
-public interface WritableByteArrayComparable extends Writable,
-    Comparable<byte[]> {
+public interface WritableByteArrayComparable extends Writable, Comparable<byte[]> {
   // Not methods, just tie the two interfaces together.
 }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/package-info.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/package-info.java?rev=790043&r1=790042&r2=790043&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/package-info.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/package-info.java Wed Jul  1 05:32:54 2009
@@ -20,8 +20,9 @@
 /**Provides row-level filters applied to HRegion scan results during calls to
  * {@link org.apache.hadoop.hbase.client.ResultScanner#next()}. 
 
-<p>Since HBase 0.20.0, {@link org.apache.hadoop.hbase.filter.Filter} is the new Interface used filtering.
-It replaces the deprecated {@link org.apache.hadoop.hbase.filter.RowFilterInterface}.
+<p>Since HBase 0.20.0, {@link org.apache.hadoop.hbase.filter.Filter} is the new
+Interface used filtering.  It replaces the deprecated
+{@link org.apache.hadoop.hbase.filter.RowFilterInterface}.
 Filters run the extent of a table unless you wrap your filter in a
 {@link org.apache.hadoop.hbase.filter.WhileMatchFilter}.
 The latter returns as soon as the filter stops matching.

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java?rev=790043&r1=790042&r2=790043&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java Wed Jul  1 05:32:54 2009
@@ -143,6 +143,7 @@
     addToMap(PageFilter.class, code++);
     addToMap(InclusiveStopFilter.class, code++);
     addToMap(ColumnCountGetFilter.class, code++);
+    addToMap(ValueFilter.class, code++);
   }
   
   private Class<?> declaredClass;

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestFilterSet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestFilterSet.java?rev=790043&r1=790042&r2=790043&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestFilterSet.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestFilterSet.java Wed Jul  1 05:32:54 2009
@@ -27,6 +27,7 @@
 import java.util.Set;
 
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.KeyValue;
 
 
 import junit.framework.TestCase;
@@ -67,8 +68,26 @@
     filterMPONE.reset();
     assertFalse(filterMPONE.filterAllRemaining());
     byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
+    for (int i = 0; i < MAX_PAGES; i++) {
+      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
+        Bytes.toBytes(i));
+      assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+    }
+    rowkey = Bytes.toBytes("z");
+    for (int i = 0; i < MAX_PAGES - 1; i++) {
+      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
+        Bytes.toBytes(i));
+      assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+    }
     assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
-    
+    // Should fail here
+    KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
+    assertTrue(Filter.ReturnCode.SKIP == filterMPONE.filterKeyValue(kv));
+
+    // Both filters in Set should be satisfied by now
+    assertTrue(filterMPONE.filterRow());
   }
 
   /**
@@ -81,6 +100,36 @@
     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
     Filter filterMPALL =
       new FilterSet(FilterSet.Operator.MUST_PASS_ALL, filters);
+    /* Filter must do all below steps:
+     * <ul>
+     * <li>{@link #reset()}</li>
+     * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
+     * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
+     * if false, we will also call</li>
+     * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
+     * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
+     * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
+     * </li>
+     * </ul>
+    */
+    filterMPALL.reset();
+    assertFalse(filterMPALL.filterAllRemaining());
+    byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
+    for (int i = 0; i < MAX_PAGES - 1; i++) {
+      assertFalse(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
+      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
+        Bytes.toBytes(i));
+      assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv));
+    }
+    filterMPALL.reset();
+    rowkey = Bytes.toBytes("z");
+    assertTrue(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
+    // Should fail here; row should be filtered out.
+    KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
+    assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterKeyValue(kv));
+
+    // Both filters in Set should be satisfied by now
+    assertTrue(filterMPALL.filterRow());
   }
 
   /**

Copied: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java (from r789818, hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestInclusiveStopRowFilter.java)
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java?p2=hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java&p1=hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestInclusiveStopRowFilter.java&r1=789818&r2=790043&rev=790043&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestInclusiveStopRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java Wed Jul  1 05:32:54 2009
@@ -33,19 +33,19 @@
 /**
  * Tests the inclusive stop row filter
  */
-public class DisabledTestInclusiveStopRowFilter extends TestCase {
+public class TestInclusiveStopFilter extends TestCase {
   private final byte [] STOP_ROW = Bytes.toBytes("stop_row");
   private final byte [] GOOD_ROW = Bytes.toBytes("good_row");
   private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz");
-  
-  RowFilterInterface mainFilter;
+
+  Filter mainFilter;
 
   @Override
   protected void setUp() throws Exception {
     super.setUp();
-    mainFilter = new InclusiveStopRowFilter(STOP_ROW);
+    mainFilter = new InclusiveStopFilter(STOP_ROW);
   }
-  
+
   /**
    * Tests identification of the stop row
    * @throws Exception
@@ -53,7 +53,7 @@
   public void testStopRowIdentification() throws Exception {
     stopRowTests(mainFilter);
   }
-  
+
   /**
    * Tests serialization
    * @throws Exception
@@ -65,30 +65,27 @@
     mainFilter.write(out);
     out.close();
     byte[] buffer = stream.toByteArray();
-    
+
     // Recompose mainFilter.
     DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
-    RowFilterInterface newFilter = new InclusiveStopRowFilter();
+    Filter newFilter = new InclusiveStopFilter();
     newFilter.readFields(in);
-    
+
     // Ensure the serialization preserved the filter by running a full test.
     stopRowTests(newFilter);
   }
-  
-  private void stopRowTests(RowFilterInterface filter) throws Exception {
-    assertFalse("Filtering on " + Bytes.toString(GOOD_ROW), filter.filterRowKey(GOOD_ROW));
-    assertFalse("Filtering on " + Bytes.toString(STOP_ROW), filter.filterRowKey(STOP_ROW));
-    assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW), filter.filterRowKey(PAST_STOP_ROW));
-    
-    assertFalse("Filtering on " + Bytes.toString(GOOD_ROW), filter.filterColumn(GOOD_ROW, null, 
-      null));
-    assertFalse("Filtering on " + Bytes.toString(STOP_ROW), filter.filterColumn(STOP_ROW, null, null));
-    assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW), filter.filterColumn(PAST_STOP_ROW, 
-      null, null));
+
+  private void stopRowTests(Filter filter) throws Exception {
+    assertFalse("Filtering on " + Bytes.toString(GOOD_ROW),
+      filter.filterRowKey(GOOD_ROW, 0, GOOD_ROW.length));
+    assertFalse("Filtering on " + Bytes.toString(STOP_ROW),
+      filter.filterRowKey(STOP_ROW, 0, STOP_ROW.length));
+    assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW),
+      filter.filterRowKey(PAST_STOP_ROW, 0, PAST_STOP_ROW.length));
 
     assertFalse("FilterAllRemaining", filter.filterAllRemaining());
-    assertFalse("FilterNotNull", filter.filterRow((List<KeyValue>)null));
-    
-    assertFalse("Filter a null", filter.filterRowKey(null));
+    assertFalse("FilterNotNull", filter.filterRow());
+
+    assertFalse("Filter a null", filter.filterRowKey(null, 0, 0));
   }
-}
+}
\ No newline at end of file

Copied: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestValueFilter.java (from r789818, hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java)
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestValueFilter.java?p2=hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestValueFilter.java&p1=hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java&r1=789818&r2=790043&rev=790043&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestValueFilter.java Wed Jul  1 05:32:54 2009
@@ -31,12 +31,12 @@
 import junit.framework.TestCase;
 
 /**
- * Tests the stop row filter
+ * Tests the value filter
  */
-public class TestColumnValueFilter extends TestCase {
-
+public class TestValueFilter extends TestCase {
   private static final byte[] ROW = Bytes.toBytes("test");
-  private static final byte[] COLUMN = Bytes.toBytes("test:foo");
+  private static final byte[] COLUMN_FAMILY = Bytes.toBytes("test");
+  private static final byte [] COLUMN_QUALIFIER = Bytes.toBytes("foo");
   private static final byte[] VAL_1 = Bytes.toBytes("a");
   private static final byte[] VAL_2 = Bytes.toBytes("ab");
   private static final byte[] VAL_3 = Bytes.toBytes("abc");
@@ -48,56 +48,76 @@
   private static final String QUICK_SUBSTR = "quick";
   private static final String QUICK_REGEX = ".+quick.+";
 
-  private RowFilterInterface basicFilterNew() {
-    return new ColumnValueFilter(COLUMN,
-        ColumnValueFilter.CompareOp.GREATER_OR_EQUAL, VAL_2);
+  Filter basicFilter;
+  Filter substrFilter;
+  Filter regexFilter;
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    basicFilter = basicFilterNew();
+    substrFilter = substrFilterNew();
+    regexFilter = regexFilterNew();
+  }
+
+  private Filter basicFilterNew() {
+    return new ValueFilter(COLUMN_FAMILY, COLUMN_QUALIFIER,
+      ValueFilter.CompareOp.GREATER_OR_EQUAL, VAL_2);
   }
 
-  private RowFilterInterface substrFilterNew() {
-    return new ColumnValueFilter(COLUMN, ColumnValueFilter.CompareOp.EQUAL,
+  private Filter substrFilterNew() {
+    return new ValueFilter(COLUMN_FAMILY, COLUMN_QUALIFIER,
+      ValueFilter.CompareOp.EQUAL,
       new SubstringComparator(QUICK_SUBSTR));
   }
 
-  private RowFilterInterface regexFilterNew() {
-    return new ColumnValueFilter(COLUMN, ColumnValueFilter.CompareOp.EQUAL,
+  private Filter regexFilterNew() {
+    return new ValueFilter(COLUMN_FAMILY, COLUMN_QUALIFIER,
+      ValueFilter.CompareOp.EQUAL,
       new RegexStringComparator(QUICK_REGEX));
   }
 
-  private void basicFilterTests(RowFilterInterface filter)
+  private void basicFilterTests(Filter filter)
       throws Exception {
-    assertTrue("basicFilter1", filter.filterColumn(ROW, 0, ROW.length,
-      COLUMN, 0, COLUMN.length, VAL_1, 0, VAL_1.length));
-    assertFalse("basicFilter2", filter.filterColumn(ROW, 0, ROW.length,
-      COLUMN, 0, COLUMN.length, VAL_2, 0, VAL_2.length));
-    assertFalse("basicFilter3", filter.filterColumn(ROW, 0, ROW.length,
-      COLUMN, 0, COLUMN.length, VAL_3, 0, VAL_3.length));
-    assertFalse("basicFilter4", filter.filterColumn(ROW, 0, ROW.length,
-      COLUMN, 0, COLUMN.length, VAL_4, 0, VAL_4.length));
+    KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
+    assertFalse("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
+    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
+    assertTrue("basicFilter2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
+    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3);
+    assertTrue("basicFilter3", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
+    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4);
+    assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     assertFalse("basicFilterAllRemaining", filter.filterAllRemaining());
-    assertFalse("basicFilterNotNull", filter.filterRow((List<KeyValue>)null));
+    assertTrue("basicFilterNotNull", filter.filterRow());
   }
 
-  private void substrFilterTests(RowFilterInterface filter) 
+  private void substrFilterTests(Filter filter) 
       throws Exception {
-    assertFalse("substrTrue", filter.filterColumn(ROW, 0, ROW.length,
-        COLUMN, 0, COLUMN.length, FULLSTRING_1, 0, FULLSTRING_1.length));
-    assertTrue("substrFalse", filter.filterColumn(ROW, 0, ROW.length,
-        COLUMN, 0, COLUMN.length, FULLSTRING_2, 0, FULLSTRING_2.length));
+    KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+      FULLSTRING_1);
+    assertTrue("substrTrue",
+      filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
+    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+      FULLSTRING_2);
+    assertFalse("substrFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     assertFalse("substrFilterAllRemaining", filter.filterAllRemaining());
-    assertFalse("substrFilterNotNull", filter.filterRow((List<KeyValue>)null));
+    assertTrue("substrFilterNotNull", filter.filterRow());
   }
 
-  private void regexFilterTests(RowFilterInterface filter) 
+  private void regexFilterTests(Filter filter) 
       throws Exception {
-    assertFalse("regexTrue", filter.filterColumn(ROW, 0, ROW.length,
-        COLUMN, 0, COLUMN.length, FULLSTRING_1, 0, FULLSTRING_1.length));
-    assertTrue("regexFalse", filter.filterColumn(ROW, 0, ROW.length,
-        COLUMN, 0, COLUMN.length, FULLSTRING_2, 0, FULLSTRING_2.length));
+    KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+      FULLSTRING_1);
+    assertTrue("regexTrue",
+      filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
+    kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+      FULLSTRING_2);
+    assertFalse("regexFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
-    assertFalse("regexFilterNotNull", filter.filterRow((List<KeyValue>)null));
-  }
-
-  private RowFilterInterface serializationTest(RowFilterInterface filter)
+    assertTrue("regexFilterNotNull", filter.filterRow());
+  }    
+                 
+  private Filter serializationTest(Filter filter)
       throws Exception {
     // Decompose filter to bytes.
     ByteArrayOutputStream stream = new ByteArrayOutputStream();
@@ -109,24 +129,12 @@
     // Recompose filter.
     DataInputStream in =
       new DataInputStream(new ByteArrayInputStream(buffer));
-    RowFilterInterface newFilter = new ColumnValueFilter();
+    Filter newFilter = new ValueFilter();
     newFilter.readFields(in);
   
     return newFilter;
   }
 
-  RowFilterInterface basicFilter;
-  RowFilterInterface substrFilter;
-  RowFilterInterface regexFilter;
-
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    basicFilter = basicFilterNew();
-    substrFilter = substrFilterNew();
-    regexFilter = regexFilterNew();
-  }
-
   /**
    * Tests identification of the stop row
    * @throws Exception
@@ -135,19 +143,18 @@
     basicFilterTests(basicFilter);
     substrFilterTests(substrFilter);
     regexFilterTests(regexFilter);
-  }
+  }                               
 
   /**
    * Tests serialization
    * @throws Exception
-   */
+   */                       
   public void testSerialization() throws Exception {
-    RowFilterInterface newFilter = serializationTest(basicFilter);
+    Filter newFilter = serializationTest(basicFilter);
     basicFilterTests(newFilter);
     newFilter = serializationTest(substrFilter);
     substrFilterTests(newFilter);
     newFilter = serializationTest(regexFilter);
     regexFilterTests(newFilter);
-  }
-
+  }                   
 }