hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Allan Yang (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-18368) Filters with OR do not work
Date Sun, 16 Jul 2017 14:55:00 GMT

    [ https://issues.apache.org/jira/browse/HBASE-18368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16088947#comment-16088947
] 

Allan Yang commented on HBASE-18368:
------------------------------------

[~anoop.hbase] you are right, matchingRowColumn have already checked faimiles.  Updated a
v3 patch. Only the logic in return code with NEXT_ROW need to be changed. As I said before,
the root cause of this bug is that  HBASE-17678 introduced a list to record previous return
code and cell.
  If the filter list is MUST_PASS_ONE and one filter has returned NEXT_ROW, if the next cell
arrivals at the filter has the same row as previous one, it can bypass the filter and skip
this cell since it want a "next row". 
 But, this is not the case for FamilyFilter.  HBASE-13122 has introduced a optimization for
it. Instead of returning "SKIP" when family not matching, it return "NEXT_ROW".

So, my way to fix this bug is that, we should check family before we can bypass the filter.


> Filters with OR do not work
> ---------------------------
>
>                 Key: HBASE-18368
>                 URL: https://issues.apache.org/jira/browse/HBASE-18368
>             Project: HBase
>          Issue Type: Bug
>          Components: Filters
>    Affects Versions: 3.0.0, 2.0.0-alpha-1
>            Reporter: Peter Somogyi
>            Assignee: Allan Yang
>            Priority: Critical
>         Attachments: HBASE-18368.branch-1.patch, HBASE-18368.branch-1.v2.patch, HBASE-18368.branch-1.v3.patch
>
>
> 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.
> {code:java|title=Test code}
>   @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);
>     }
>   }
> {code}
> {noformat:title=Output}
> FilterList OR (2/2): [FamilyFilter (EQUAL, cf1), FamilyFilter (EQUAL, cf2)]
> keyvalues={0/cf1:col_a/1499852754957/Put/vlen=4/seqid=0}
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message