Details
-
Bug
-
Status: Resolved
-
Critical
-
Resolution: Fixed
-
2.4.11
-
None
-
Reviewed
Description
I created test table with two column families by hbase shell:
create 'test_table2', 'f1', 'f2' put 'test_table2', '1', 'f1:col1', 'a1' put 'test_table2', '1', 'f2:col2', 'a2' put 'test_table2', '2', 'f1:col1', 'b1' put 'test_table2', '2', 'f2:col2', 'b2'
The table contains of two rows (rowkeys '1' and '2'), tested FuzzyRowFilter selects first row,
SingleColumnValueFilter selects no rows, combination of both filters evaluated with
MUST_PASS_ONE operator returns surprisingly whole table. I prepared java examples to
show this strange behavior.
Code snippet below doesn't work as expected:
try (Table table = connection.getTable(TableName.valueOf("test_table2"))) { Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("f1")); scan.addFamily(Bytes.toBytes("f2")); Filter fuzzyRowFilter = new FuzzyRowFilter(List.of(new Pair<>(Bytes.toBytes("1"), new byte[] { 0x00 }))); scan.setFilter(fuzzyRowFilter); System.out.println("result of fuzzy filter:"); for (Result r : table.getScanner(scan)) { System.out.println(Bytes.toString(r.getRow())); } System.out.println("result of single column value filter:"); Filter singleColumnValueFilter = new SingleColumnValueFilter(Bytes.toBytes("f2"), Bytes.toBytes("col2"), CompareOperator.EQUAL, Bytes.toBytes("x")); scan.setFilter(singleColumnValueFilter); for (Result r : table.getScanner(scan)) { System.out.println(Bytes.toString(r.getRow())); } System.out.println("result of fuzzy or single column value filters:"); FilterList filterList = new FilterList(Operator.MUST_PASS_ONE); filterList.addFilter(fuzzyRowFilter); filterList.addFilter(singleColumnValueFilter); scan.setFilter(filterList); for (Result r : table.getScanner(scan)) { System.out.println(Bytes.toString(r.getRow())); } }
Expected result in my opinion is:
result of fuzzy filter:
1
result of single column value filter:
result of fuzzy or single column value filters:
1
But i am getting (NOT OK):
result of fuzzy filter:
1
result of single column value filter:
result of fuzzy or single column value filters:
1
2
For tables with one column family or commentig out the line scan.addFamily(Bytes.toBytes("f1")) filter list evaluation is working OK. Similar example with PrefixFilter is working like a charm:
try (Table table = connection.getTable(TableName.valueOf("test_table2"))) { Scan scan = new Scan(); scan.addFamily(Bytes.toBytes("f1")); scan.addFamily(Bytes.toBytes("f2")); Filter prefixFilter = new PrefixFilter(Bytes.toBytes("1")); scan.setFilter(prefixFilter); System.out.println("result of prefix filter:"); for (Result r : table.getScanner(scan)) { System.out.println(Bytes.toString(r.getRow())); } System.out.println("result of single column value filter:"); Filter singleColumnValueFilter = new SingleColumnValueFilter(Bytes.toBytes("f2"), Bytes.toBytes("col2"), CompareOperator.EQUAL, Bytes.toBytes("x")); scan.setFilter(singleColumnValueFilter); for (Result r : table.getScanner(scan)) { System.out.println(Bytes.toString(r.getRow())); } System.out.println("result of prefix or single column value filters"); FilterList filterList = new FilterList(Operator.MUST_PASS_ONE); filterList.addFilter(prefixFilter); filterList.addFilter(singleColumnValueFilter); scan.setFilter(filterList); for (Result r : table.getScanner(scan)) { System.out.println(Bytes.toString(r.getRow())); } }
Result OK:
result of prefix filter:
1
result of single column value filter:
result of prefix or single column value filters
1
Attachments
Issue Links
- links to