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

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

{quote}
I ran the test from your previous patch in master branch based on commit 
353627b39de73020dd2448b54c0f13f6902b19bf .
It passed.
{quote}
That is very very strange. Updated a patch for master branch, Peter's original 
test is include(added a assert to fail the test, since Peter's original one is 
just to print the result). If FilterList.java is not patched, this test and my 
UT should fail. You can try the patch, [~tedyu].
But, I still think we should not include the original test, In order to just to 
test the function of filters, starting a mini cluster, and then write some data 
is too 'heavy'.

> 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, 
> HBASE-18368.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)

Reply via email to