You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Boris (Jira)" <ji...@apache.org> on 2022/04/22 07:08:00 UTC

[jira] [Updated] (HBASE-26967) FilterList with FuzzyRowFilter and SingleColumnValueFilter evaluated with operator MUST_PASS_ONE doesn't work as expected

     [ https://issues.apache.org/jira/browse/HBASE-26967?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Boris updated HBASE-26967:
--------------------------
    Description: 
I created test table with two column families by hbase shell:

 
{code:java}
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' {code}
 

 

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:

 
{code:java}
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()));
    }
} {code}
Expected result in my opinion is:

 
{quote}result of fuzzy filter:
1
result of single column value filter:
result of fuzzy or single column value filters:
1
{quote}
But i am getting (NOT OK):
{quote}result of fuzzy filter:
1
result of single column value filter:
result of fuzzy or single column value filters:
1
2
{quote}
 

For tables with one column family or commentig out the line _{color:#000000}scan{color}.addFamily({color:#000000}Bytes{color}.toBytes({color:#067d17}"f1"{color}))_ filter list evaluation is working OK. Similar example with PrefixFilter is working like a charm:

 
{code:java}
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()));
    }
} {code}
 

 

Result OK{_}:{_}
{quote}result of prefix filter:
1
result of single column value filter:
result of prefix or single column value filters
1
{quote}

  was:
I created test table with two column families:
{quote}_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'_
{quote}
code snippet below doesn't work as expected:
{quote}_{color:#0033b3}try {color}({color:#000000}Table table {color}= {color:#000000}connection{color}.getTable({color:#000000}TableName{color}.valueOf({color:#067d17}"test_table2"{color}))) {_
 _{color:#000000}Scan scan {color}= {color:#0033b3}new {color}Scan();_
 _{color:#000000}scan{color}.addFamily({color:#000000}Bytes{color}.toBytes({color:#067d17}"f1"{color}));_
 _{color:#000000}scan{color}.addFamily({color:#000000}Bytes{color}.toBytes({color:#067d17}"f2"{color}));_
 _{color:#000000}scan{color}.setFilter({color:#0033b3}new {color}FuzzyRowFilter({color:#000000}List{color}.of({color:#0033b3}new {color}Pair<>({color:#000000}Bytes{color}.toBytes({color:#067d17}"1"{color}), {color:#0033b3}new byte{color}[] { {color:#1750eb}0x00 {color}}))));_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#067d17}"result of fuzzy filter:"{color});_
 _{color:#0033b3}for {color}({color:#000000}Result r {color}: {color:#000000}table{color}.getScanner({color:#000000}scan{color})) {_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#000000}Bytes{color}.toString({color:#000000}r{color}.getRow()));_
 _}_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#067d17}"result of single column value filter:"{color});_
 _{color:#000000}scan{color}.setFilter(_
 _{color:#0033b3}new {color}SingleColumnValueFilter({color:#000000}Bytes{color}.toBytes({color:#067d17}"f2"{color}), {color:#000000}Bytes{color}.toBytes({color:#067d17}"col2"{color}), {color:#000000}CompareOperator{color}.{color:#871094}EQUAL{color}, {color:#000000}Bytes{color}.toBytes({color:#067d17}"x"{color})));_
 _{color:#0033b3}for {color}({color:#000000}Result r {color}: {color:#000000}table{color}.getScanner({color:#000000}scan{color})) {_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#000000}Bytes{color}.toString({color:#000000}r{color}.getRow()));_
 _}_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#067d17}"result of fuzzy or single column value filters:"{color});_
 _{color:#000000}FilterList filterList {color}= {color:#0033b3}new {color}FilterList({color:#000000}Operator{color}.{color:#871094}MUST_PASS_ONE{color});_
 _{color:#000000}filterList{color}.addFilter({color:#0033b3}new {color}FuzzyRowFilter({color:#000000}List{color}.of({color:#0033b3}new {color}Pair<>({color:#000000}Bytes{color}.toBytes({color:#067d17}"1"{color}), {color:#0033b3}new byte{color}[] { {color:#1750eb}0x00 {color}}))));_
 _{color:#000000}filterList{color}.addFilter(_
 _{color:#0033b3}new {color}SingleColumnValueFilter({color:#000000}Bytes{color}.toBytes({color:#067d17}"f2"{color}), {color:#000000}Bytes{color}.toBytes({color:#067d17}"col2"{color}), {color:#000000}CompareOperator{color}.{color:#871094}EQUAL{color}, {color:#000000}Bytes{color}.toBytes({color:#067d17}"x"{color})));_
 _{color:#000000}scan{color}.setFilter({color:#000000}filterList{color});_
 _{color:#0033b3}for {color}({color:#000000}Result r {color}: {color:#000000}table{color}.getScanner({color:#000000}scan{color})) {_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#000000}Bytes{color}.toString({color:#000000}r{color}.getRow()));_
 _}_
_}_
{quote}
Expected result in my opinion is:
{quote}result of fuzzy filter:
1
result of single column value filter:
result of fuzzy or single column value filters:
1
{quote}
But i got (NOT OK):
{quote}result of fuzzy filter:
1
result of single column value filter:
result of fuzzy or single column value filters:
1
2
{quote}
For tables with one column family or commentig out line _{color:#000000}scan{color}.addFamily({color:#000000}Bytes{color}.toBytes({color:#067d17}"f1"{color}))_ filter list evaluation is working OK. Similar example with PrefixFilter is working like a charm:
{quote}_{color:#0033b3}try {color}({color:#000000}Table table {color}= {color:#000000}connection{color}.getTable({color:#000000}TableName{color}.valueOf({color:#067d17}"test_table2"{color}))) {_
 _{color:#000000}Scan scan {color}= {color:#0033b3}new {color}Scan();_
 _{color:#000000}scan{color}.addFamily({color:#000000}Bytes{color}.toBytes({color:#067d17}"f1"{color}));_
 _{color:#000000}scan{color}.addFamily({color:#000000}Bytes{color}.toBytes({color:#067d17}"f2"{color}));_
 _{color:#000000}scan{color}.setFilter({color:#0033b3}new {color}PrefixFilter({color:#000000}Bytes{color}.toBytes({color:#067d17}"1"{color})));_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#067d17}"result of prefix filter:"{color});_
 _{color:#0033b3}for {color}({color:#000000}Result r {color}: {color:#000000}table{color}.getScanner({color:#000000}scan{color})) {_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#000000}Bytes{color}.toString({color:#000000}r{color}.getRow()));_
 _}_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#067d17}"result of single column value filter:"{color});_
 _{color:#000000}scan{color}.setFilter(_
 _{color:#0033b3}new {color}SingleColumnValueFilter({color:#000000}Bytes{color}.toBytes({color:#067d17}"f2"{color}), {color:#000000}Bytes{color}.toBytes({color:#067d17}"col2"{color}), {color:#000000}CompareOperator{color}.{color:#871094}EQUAL{color}, {color:#000000}Bytes{color}.toBytes({color:#067d17}"x"{color})));_
 _{color:#0033b3}for {color}({color:#000000}Result r {color}: {color:#000000}table{color}.getScanner({color:#000000}scan{color})) {_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#000000}Bytes{color}.toString({color:#000000}r{color}.getRow()));_
 _}_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#067d17}"result of prefix or single column value filters"{color});_
 _{color:#000000}FilterList filterList {color}= {color:#0033b3}new {color}FilterList({color:#000000}Operator{color}.{color:#871094}MUST_PASS_ONE{color});_
 _{color:#000000}filterList{color}.addFilter({color:#0033b3}new {color}PrefixFilter({color:#000000}Bytes{color}.toBytes({color:#067d17}"1"{color})));_
 _{color:#000000}filterList{color}.addFilter(_
 _{color:#0033b3}new {color}SingleColumnValueFilter({color:#000000}Bytes{color}.toBytes({color:#067d17}"f2"{color}), {color:#000000}Bytes{color}.toBytes({color:#067d17}"col2"{color}), {color:#000000}CompareOperator{color}.{color:#871094}EQUAL{color}, {color:#000000}Bytes{color}.toBytes({color:#067d17}"x"{color})));_
 _{color:#000000}scan{color}.setFilter({color:#000000}filterList{color});_
 _{color:#0033b3}for {color}({color:#000000}Result r {color}: {color:#000000}table{color}.getScanner({color:#000000}scan{color})) {_
 _{color:#000000}System{color}.{color:#871094}out{color}.println({color:#000000}Bytes{color}.toString({color:#000000}r{color}.getRow()));_
 _}_
_}_
{quote}
Result OK{_}:{_}
{quote}result of prefix filter:
1
result of single column value filter:
result of prefix or single column value filters
1
{quote}


> FilterList with FuzzyRowFilter and SingleColumnValueFilter evaluated with operator MUST_PASS_ONE doesn't work as expected
> -------------------------------------------------------------------------------------------------------------------------
>
>                 Key: HBASE-26967
>                 URL: https://issues.apache.org/jira/browse/HBASE-26967
>             Project: HBase
>          Issue Type: Bug
>          Components: Filters
>    Affects Versions: 2.4.11
>            Reporter: Boris
>            Priority: Major
>
> I created test table with two column families by hbase shell:
>  
> {code:java}
> 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' {code}
>  
>  
> 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:
>  
> {code:java}
> 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()));
>     }
> } {code}
> Expected result in my opinion is:
>  
> {quote}result of fuzzy filter:
> 1
> result of single column value filter:
> result of fuzzy or single column value filters:
> 1
> {quote}
> But i am getting (NOT OK):
> {quote}result of fuzzy filter:
> 1
> result of single column value filter:
> result of fuzzy or single column value filters:
> 1
> 2
> {quote}
>  
> For tables with one column family or commentig out the line _{color:#000000}scan{color}.addFamily({color:#000000}Bytes{color}.toBytes({color:#067d17}"f1"{color}))_ filter list evaluation is working OK. Similar example with PrefixFilter is working like a charm:
>  
> {code:java}
> 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()));
>     }
> } {code}
>  
>  
> Result OK{_}:{_}
> {quote}result of prefix filter:
> 1
> result of single column value filter:
> result of prefix or single column value filters
> 1
> {quote}



--
This message was sent by Atlassian Jira
(v8.20.7#820007)