Details
-
Sub-task
-
Status: Resolved
-
Critical
-
Resolution: Fixed
-
3.0.0-alpha-1, 2.0.0-alpha-1
-
None
-
None
-
Incompatible change
Description
Scan gives back incomplete list if multiple filters are combined with OR / MUST_PASS_ONE.
Using 2 FamilyFilters in a FilterList using MUST_PASS_ONE operator will give back results for only the first Filter.
@Test public void testFiltersWithOr() throws Exception { TableName tn = TableName.valueOf("MyTest"); Table table = utility.createTable(tn, new String[] {"cf1", "cf2"}); byte[] CF1 = Bytes.toBytes("cf1"); byte[] CF2 = Bytes.toBytes("cf2"); Put put1 = new Put(Bytes.toBytes("0")); put1.addColumn(CF1, Bytes.toBytes("col_a"), Bytes.toBytes(0)); table.put(put1); Put put2 = new Put(Bytes.toBytes("0")); put2.addColumn(CF2, Bytes.toBytes("col_b"), Bytes.toBytes(0)); table.put(put2); FamilyFilter filterCF1 = new FamilyFilter(CompareFilter.CompareOp.EQUAL, new BinaryComparator(CF1)); FamilyFilter filterCF2 = new FamilyFilter(CompareFilter.CompareOp.EQUAL, new BinaryComparator(CF2)); FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE); filterList.addFilter(filterCF1); filterList.addFilter(filterCF2); Scan scan = new Scan(); scan.setFilter(filterList); ResultScanner scanner = table.getScanner(scan); System.out.println(filterList); for (Result rr = scanner.next(); rr != null; rr = scanner.next()) { System.out.println(rr); } }
FilterList OR (2/2): [FamilyFilter (EQUAL, cf1), FamilyFilter (EQUAL, cf2)] keyvalues={0/cf1:col_a/1499852754957/Put/vlen=4/seqid=0}
Attachments
Attachments
Issue Links
- blocks
-
YARN-7213 [Umbrella] Test and validate HBase-2.0.x with Atsv2
- Resolved
- is related to
-
YARN-7213 [Umbrella] Test and validate HBase-2.0.x with Atsv2
- Resolved
- relates to
-
HBASE-18957 add test that confirms 2 FamilyFilters in a FilterList using MUST_PASS_ONE operator will return results that match either of the FamilyFilters and revert as needed to make it pass.
- Resolved
- links to