[jira] [Comment Edited] (HBASE-18368) Filters with OR do not work

2017-07-19 Thread Zheng Hu (JIRA)

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

Zheng Hu edited comment on HBASE-18368 at 7/19/17 6:19 AM:
---

After reconsider the patch , I think it may introduce another BUG.   I made the 
following UT which was failed after applied the latest patch , [~allan163],  
[~chia7712] , [~tedyu]. 
{code}
 private static class RowCountFilter extends FilterBase {
private int rowCount = 0;

public ReturnCode filterKeyValue(Cell v) throws IOException {
  rowCount++;
  return ReturnCode.NEXT_ROW;
}

public int getRowCount() {
  return rowCount;
}
  }

  @Test
  public void testRowCountFilter() throws IOException {
KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam1"), 
Bytes.toBytes("a"), 1,
Bytes.toBytes("value"));
KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam2"), 
Bytes.toBytes("a"), 2,
Bytes.toBytes("value"));
RowCountFilter rowCountFilter = new RowCountFilter();
FilterList filter = new FilterList(Operator.MUST_PASS_ONE, rowCountFilter);

filter.filterKeyValue(kv1);
filter.filterKeyValue(kv2);

Assert.assertEquals(rowCountFilter.getRowCount(), 1);
  }
{code}


was (Author: openinx):
After reconsider the patch , I think it may introduce another BUG.   I made the 
following UT which was failed after applied the latest patch , [~allan163],  
[~chia7712] . 
{code}
 private static class RowCountFilter extends FilterBase {
private int rowCount = 0;

public ReturnCode filterKeyValue(Cell v) throws IOException {
  rowCount++;
  return ReturnCode.NEXT_ROW;
}

public int getRowCount() {
  return rowCount;
}
  }

  @Test
  public void testRowCountFilter() throws IOException {
KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam1"), 
Bytes.toBytes("a"), 1,
Bytes.toBytes("value"));
KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam2"), 
Bytes.toBytes("a"), 2,
Bytes.toBytes("value"));
RowCountFilter rowCountFilter = new RowCountFilter();
FilterList filter = new FilterList(Operator.MUST_PASS_ONE, rowCountFilter);

filter.filterKeyValue(kv1);
filter.filterKeyValue(kv2);

Assert.assertEquals(rowCountFilter.getRowCount(), 1);
  }
{code}

> 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)


[jira] [Comment Edited] (HBASE-18368) Filters with OR do not work

2017-07-17 Thread Zheng Hu (JIRA)

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

Zheng Hu edited comment on HBASE-18368 at 7/17/17 7:56 AM:
---

Sorry for my delay,  Actually,  I don't think [~allan163]'s patch is the 
correct solution.   In issue HBASE-18160 & HBASE-17678 ,  I summarized that 
FilterList with MUST_PASS_ONE should observe The Mininal Step Rule (filter list 
with MUST_PASS_ONE should choose the mininal step among filters in filter list. 
Let's call it: The Mininal Step Rule).And the bug that Peter presented is 
caused by the violation of  The Minnal Step Rule.

See that:

{code}
switch (localRC) {
case INCLUDE:
  if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
rc = ReturnCode.INCLUDE;
  }
  transformed = filter.transformCell(transformed);
  break;
case INCLUDE_AND_NEXT_COL:
  rc = ReturnCode.INCLUDE_AND_NEXT_COL;
  transformed = filter.transformCell(transformed);
  // must continue here to evaluate all filters
  break;
case NEXT_ROW:
  break;  <  That's the key point
case SKIP: 
  break;  <  That's the key point
case NEXT_COL:
  break;  <  That's the key point
case SEEK_NEXT_USING_HINT:
  break;  <  That's the key point
default:
  throw new IllegalStateException("Received code is not valid.");
}
{code} 


For cell with {cf: cf2, qualifier: col_b}, the correct logic is : 
1. filterCF1 should return NEXT_ROW ; 
2. filterCF2 should return INCLUDE  ;
3. According to The Minnal Step Rule, filterList should return INCLUDE ; 

Back to our code: 
1. filterCF1 return NEXT_ROW, and the for loop is interrupted by the *break* ; 
2. So filterCF2 would be skipped by our code. 
3. Finally, filterList will return SKIP. 



was (Author: openinx):
Sorry for my delay,  Actually,  I don't think [~allan163]'s patch is the 
correct solution.   In issue HBASE-18160 & HBASE-17678 ,  I summarized that 
FilterList with MUST_PASS_ONE should observe The Mininal Step Rule (filter list 
with MUST_PASS_ONE should choose the mininal step among filters in filter list. 
Let's call it: The Mininal Step Rule).And the bug that Peter presented is 
caused by the violation of  The Minnal Step Rule.

See that:

{code}
switch (localRC) {
case INCLUDE:
  if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
rc = ReturnCode.INCLUDE;
  }
  transformed = filter.transformCell(transformed);
  break;
case INCLUDE_AND_NEXT_COL:
  rc = ReturnCode.INCLUDE_AND_NEXT_COL;
  transformed = filter.transformCell(transformed);
  // must continue here to evaluate all filters
  break;
case NEXT_ROW:
  break;  <  That's the key point
case SKIP: 
  break;  <  That's the key point
case NEXT_COL:
  break;  <  That's the key point
case SEEK_NEXT_USING_HINT:
  break;  <  That's the key point
default:
  throw new IllegalStateException("Received code is not valid.");
}
{code} 


For cell with {cf: cf2, qualifier: col_b}, the correct logic is : 
1. filterCF1 should return NEXT_ROW ; 
2. filterCF2 should return INCLUDE  ;
3. According to The Minnal Step Rule, filterList should return INCLUDE ; 

Back to our code: 
1. filterCF1 return NEXT_ROW, and the for loop is interrupted by the 
{code}break{code} ; 
2. So filterCF2 would be skipped by our code. 
3. Finally, filterList will return SKIP. 


> 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 = 

[jira] [Comment Edited] (HBASE-18368) Filters with OR do not work

2017-07-16 Thread Ted Yu (JIRA)

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

Ted Yu edited comment on HBASE-18368 at 7/16/17 3:53 PM:
-

Edit: I ran TestFilterWithScanLimits#testFiltersWithOr in master branch based 
on commit 353627b39de73020dd2448b54c0f13f6902b19bf .
It didn't have assertion so please disregard.

TestFilterList#testFamilyFilterWithMustPassOne fails without patch.



was (Author: yuzhih...@gmail.com):
I ran the test from your previous patch in master branch based on commit 
353627b39de73020dd2448b54c0f13f6902b19bf .
It passed.

Did you change the test in v3 ?

Please include Peter's test in your next patch (for master branch).

> 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)


[jira] [Comment Edited] (HBASE-18368) Filters with OR do not work

2017-07-13 Thread Ted Yu (JIRA)

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

Ted Yu edited comment on HBASE-18368 at 7/13/17 5:20 PM:
-

{code}
  public static boolean matchingRowColumn(final Cell left, final Cell right) {
if ((left.getRowLength() + left.getFamilyLength() + 
left.getQualifierLength()) != (right
.getRowLength() + right.getFamilyLength() + 
right.getQualifierLength())) {
  return false;
{code}
Family check is in the matchingColumn() method.


was (Author: yuzhih...@gmail.com):
{code}
  public static boolean matchingRowColumn(final Cell left, final Cell right) {
if ((left.getRowLength() + left.getFamilyLength() + 
left.getQualifierLength()) != (right
.getRowLength() + right.getFamilyLength() + 
right.getQualifierLength())) {
  return false;
{code}
Looks like getFamilyLength() should be taken out of the above check since 
family is not compared in the method.

> 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
>
>
> 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)