HBASE-1821 Filtering by SingleColumnValueFilter bug
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@815400 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
dd2f3d4654
commit
764cd0fa52
|
@ -24,6 +24,7 @@ Release 0.21.0 - Unreleased
|
|||
HBASE-1836 test of indexed hbase broken
|
||||
HBASE-1838 [javadoc] Add javadoc to Delete explaining behavior when no
|
||||
timestamp provided
|
||||
HBASE-1821 Filtering by SingleColumnValueFilter bug
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-1760 Cleanup TODOs in HTable
|
||||
|
|
|
@ -44,11 +44,13 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
* long value), then you can pass in your own comparator instead.
|
||||
* <p>
|
||||
* You must also specify a family and qualifier. Only the value of this column
|
||||
* will be tested. All other
|
||||
* will be tested.
|
||||
* <p>
|
||||
* To prevent the entire row from being emitted if this filter determines the
|
||||
* column does not pass (it should be filtered), wrap this filter with a
|
||||
* {@link SkipFilter}.
|
||||
* To prevent the entire row from being emitted if the column is not found
|
||||
* on a row, use {@link #setFilterIfMissing}.
|
||||
* <p>
|
||||
* Otherwise, if the column is found, the entire row will be emitted only if
|
||||
* the value passes. If the value fails, the row will be filtered out.
|
||||
* <p>
|
||||
* To filter based on the value of all scanned columns, use {@link ValueFilter}.
|
||||
*/
|
||||
|
@ -59,6 +61,9 @@ public class SingleColumnValueFilter implements Filter {
|
|||
private byte [] columnQualifier;
|
||||
private CompareOp compareOp;
|
||||
private WritableByteArrayComparable comparator;
|
||||
private boolean foundColumn = false;
|
||||
private boolean matchedColumn = false;
|
||||
private boolean filterIfMissing = false;
|
||||
|
||||
/**
|
||||
* Writable constructor, do not use.
|
||||
|
@ -109,13 +114,22 @@ public class SingleColumnValueFilter implements Filter {
|
|||
}
|
||||
|
||||
public ReturnCode filterKeyValue(KeyValue keyValue) {
|
||||
if(matchedColumn) {
|
||||
// We already found and matched the single column, all keys now pass
|
||||
return ReturnCode.INCLUDE;
|
||||
} else if(foundColumn) {
|
||||
// We found but did not match the single column, skip to next row
|
||||
return ReturnCode.NEXT_ROW;
|
||||
}
|
||||
if (!keyValue.matchingColumn(this.columnFamily, this.columnQualifier)) {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
foundColumn = true;
|
||||
if (filterColumnValue(keyValue.getBuffer(),
|
||||
keyValue.getValueOffset(), keyValue.getValueLength())) {
|
||||
return ReturnCode.NEXT_ROW;
|
||||
}
|
||||
matchedColumn = true;
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
|
@ -147,10 +161,34 @@ public class SingleColumnValueFilter implements Filter {
|
|||
}
|
||||
|
||||
public boolean filterRow() {
|
||||
return false;
|
||||
// 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
|
||||
return foundColumn ? !matchedColumn : filterIfMissing;
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
foundColumn = false;
|
||||
matchedColumn = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get whether entire row should be filtered if column is not found.
|
||||
* @return filterIfMissing true if row should be skipped if column not found,
|
||||
* false if row should be let through anyways
|
||||
*/
|
||||
public boolean getFilterIfMissing() {
|
||||
return filterIfMissing;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set whether entire row should be filtered if column is not found.
|
||||
* <p>
|
||||
* If true, the entire row will be skipped if the column is not found.
|
||||
* <p>
|
||||
* If false, the row will pass if the column is not found. This is default.
|
||||
*/
|
||||
public void setFilterIfMissing(boolean filterIfMissing) {
|
||||
this.filterIfMissing = filterIfMissing;
|
||||
}
|
||||
|
||||
public void readFields(final DataInput in) throws IOException {
|
||||
|
@ -165,6 +203,9 @@ public class SingleColumnValueFilter implements Filter {
|
|||
compareOp = CompareOp.valueOf(in.readUTF());
|
||||
comparator = (WritableByteArrayComparable) HbaseObjectWritable.readObject(in,
|
||||
null);
|
||||
foundColumn = in.readBoolean();
|
||||
matchedColumn = in.readBoolean();
|
||||
filterIfMissing = in.readBoolean();
|
||||
}
|
||||
|
||||
public void write(final DataOutput out) throws IOException {
|
||||
|
@ -173,5 +214,8 @@ public class SingleColumnValueFilter implements Filter {
|
|||
out.writeUTF(compareOp.name());
|
||||
HbaseObjectWritable.writeObject(out, comparator,
|
||||
WritableByteArrayComparable.class, null);
|
||||
out.writeBoolean(foundColumn);
|
||||
out.writeBoolean(matchedColumn);
|
||||
out.writeBoolean(filterIfMissing);
|
||||
}
|
||||
}
|
|
@ -789,6 +789,123 @@ public class TestFilter extends HBaseTestCase {
|
|||
|
||||
}
|
||||
|
||||
public void testSingleColumnValueFilter() throws IOException {
|
||||
|
||||
// From HBASE-1821
|
||||
// Desired action is to combine two SCVF in a FilterList
|
||||
// Want to return only rows that match both conditions
|
||||
|
||||
// Need to change one of the group one columns to use group two value
|
||||
Put p = new Put(ROWS_ONE[2]);
|
||||
p.add(FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]);
|
||||
this.region.put(p);
|
||||
|
||||
// Now let's grab rows that have Q_ONE[0](VALUES[0]) and Q_ONE[2](VALUES[1])
|
||||
// Since group two rows don't have these qualifiers, they will pass
|
||||
// so limiting scan to group one
|
||||
List<Filter> filters = new ArrayList<Filter>();
|
||||
filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
|
||||
CompareOp.EQUAL, VALUES[0]));
|
||||
filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2],
|
||||
CompareOp.EQUAL, VALUES[1]));
|
||||
Filter f = new FilterList(Operator.MUST_PASS_ALL, filters);
|
||||
Scan s = new Scan(ROWS_ONE[0], ROWS_TWO[0]);
|
||||
s.addFamily(FAMILIES[0]);
|
||||
s.setFilter(f);
|
||||
// Expect only one row, all qualifiers
|
||||
KeyValue [] kvs = {
|
||||
new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0])
|
||||
};
|
||||
verifyScanNoEarlyOut(s, 1, 3);
|
||||
verifyScanFull(s, kvs);
|
||||
|
||||
// In order to get expected behavior without limiting to group one
|
||||
// need to wrap SCVFs in SkipFilters
|
||||
filters = new ArrayList<Filter>();
|
||||
filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
|
||||
CompareOp.EQUAL, VALUES[0])));
|
||||
filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2],
|
||||
CompareOp.EQUAL, VALUES[1])));
|
||||
f = new FilterList(Operator.MUST_PASS_ALL, filters);
|
||||
s = new Scan(ROWS_ONE[0], ROWS_TWO[0]);
|
||||
s.addFamily(FAMILIES[0]);
|
||||
s.setFilter(f);
|
||||
// Expect same KVs
|
||||
verifyScanNoEarlyOut(s, 1, 3);
|
||||
verifyScanFull(s, kvs);
|
||||
|
||||
// More tests from HBASE-1821 for Clint and filterIfMissing flag
|
||||
|
||||
byte [][] ROWS_THREE = {
|
||||
Bytes.toBytes("rowThree-0"), Bytes.toBytes("rowThree-1"),
|
||||
Bytes.toBytes("rowThree-2"), Bytes.toBytes("rowThree-3")
|
||||
};
|
||||
|
||||
// Give row 0 and 2 QUALIFIERS_ONE[0] (VALUE[0] VALUE[1])
|
||||
// Give row 1 and 3 QUALIFIERS_ONE[1] (VALUE[0] VALUE[1])
|
||||
|
||||
KeyValue [] srcKVs = new KeyValue [] {
|
||||
new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_THREE[1], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_THREE[2], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_THREE[3], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[1])
|
||||
};
|
||||
|
||||
for(KeyValue kv : srcKVs) {
|
||||
this.region.put(new Put(kv.getRow()).add(kv));
|
||||
}
|
||||
|
||||
// Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = false
|
||||
// Expect 3 rows (0, 2, 3)
|
||||
SingleColumnValueFilter scvf = new SingleColumnValueFilter(FAMILIES[0],
|
||||
QUALIFIERS_ONE[0], CompareOp.EQUAL, VALUES[0]);
|
||||
s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
|
||||
s.addFamily(FAMILIES[0]);
|
||||
s.setFilter(scvf);
|
||||
kvs = new KeyValue [] { srcKVs[0], srcKVs[2], srcKVs[3] };
|
||||
verifyScanFull(s, kvs);
|
||||
|
||||
// Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = true
|
||||
// Expect 1 row (0)
|
||||
scvf = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
|
||||
CompareOp.EQUAL, VALUES[0]);
|
||||
scvf.setFilterIfMissing(true);
|
||||
s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
|
||||
s.addFamily(FAMILIES[0]);
|
||||
s.setFilter(scvf);
|
||||
kvs = new KeyValue [] { srcKVs[0] };
|
||||
verifyScanFull(s, kvs);
|
||||
|
||||
// Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true
|
||||
// Expect 1 row (3)
|
||||
scvf = new SingleColumnValueFilter(FAMILIES[0],
|
||||
QUALIFIERS_ONE[1], CompareOp.EQUAL, VALUES[1]);
|
||||
scvf.setFilterIfMissing(true);
|
||||
s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
|
||||
s.addFamily(FAMILIES[0]);
|
||||
s.setFilter(scvf);
|
||||
kvs = new KeyValue [] { srcKVs[3] };
|
||||
verifyScanFull(s, kvs);
|
||||
|
||||
// Add QUALIFIERS_ONE[1] to ROWS_THREE[0] with VALUES[0]
|
||||
KeyValue kvA = new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]);
|
||||
this.region.put(new Put(kvA.getRow()).add(kvA));
|
||||
|
||||
// Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true
|
||||
// Expect 1 row (3)
|
||||
scvf = new SingleColumnValueFilter(FAMILIES[0],
|
||||
QUALIFIERS_ONE[1], CompareOp.EQUAL, VALUES[1]);
|
||||
scvf.setFilterIfMissing(true);
|
||||
s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
|
||||
s.addFamily(FAMILIES[0]);
|
||||
s.setFilter(scvf);
|
||||
kvs = new KeyValue [] { srcKVs[3] };
|
||||
verifyScanFull(s, kvs);
|
||||
|
||||
}
|
||||
|
||||
private void verifyScan(Scan s, long expectedRows, long expectedKeys)
|
||||
throws IOException {
|
||||
InternalScanner scanner = this.region.getScanner(s);
|
||||
|
@ -843,8 +960,10 @@ public class TestFilter extends HBaseTestCase {
|
|||
done = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
KeyValue.COMPARATOR);
|
||||
if(results.isEmpty()) break;
|
||||
assertTrue("Scanned too many keys! Only expected " + kvs.length +
|
||||
" total but already scanned " + (results.size() + idx),
|
||||
" total but already scanned " + (results.size() + idx) +
|
||||
(results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"),
|
||||
kvs.length >= idx + results.size());
|
||||
for(KeyValue kv : results) {
|
||||
LOG.info("row=" + row + ", result=" + kv.toString() +
|
||||
|
|
|
@ -80,16 +80,20 @@ public class TestSingleColumnValueFilter extends TestCase {
|
|||
|
||||
private void basicFilterTests(Filter filter)
|
||||
throws Exception {
|
||||
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);
|
||||
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
|
||||
assertTrue("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3);
|
||||
assertTrue("basicFilter3", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
assertTrue("basicFilter2", 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());
|
||||
assertTrue("basicFilter3", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("basicFilterNotNull", filter.filterRow());
|
||||
filter.reset();
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
|
||||
assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
|
||||
assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||
assertFalse("basicFilterAllRemaining", filter.filterAllRemaining());
|
||||
assertTrue("basicFilterNotNull", filter.filterRow());
|
||||
}
|
||||
|
||||
private void substrFilterTests(Filter filter)
|
||||
|
@ -100,7 +104,7 @@ public class TestSingleColumnValueFilter extends TestCase {
|
|||
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
|
||||
FULLSTRING_2);
|
||||
assertFalse("substrFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
assertTrue("substrFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("substrFilterAllRemaining", filter.filterAllRemaining());
|
||||
assertFalse("substrFilterNotNull", filter.filterRow());
|
||||
}
|
||||
|
@ -113,7 +117,7 @@ public class TestSingleColumnValueFilter extends TestCase {
|
|||
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
|
||||
FULLSTRING_2);
|
||||
assertFalse("regexFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
assertTrue("regexFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
|
||||
assertFalse("regexFilterNotNull", filter.filterRow());
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue