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);
- }
-
+ }
}