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

Clay B. updated HBASE-22969:
----------------------------
    Description: 
Lets say you have composite key: a+b+c+d. And for simplicity assume that a,b,c, 
and d all are 4 byte integers.

Now, if you want to execute a query which is semantically same to following sql:

{{"SELECT * from table where a=1 and b > 10 and b < 20 and c > 90 and c < 100 
and d=1"}}

The only choice you have is to do client side filtering. That could be lots of 
unwanted data going through various software components and network.

Solution:

We can create a "component" comparator which takes the value of the "component" 
and its relative position in the key to pass the 'Filter' subsystem of the 
server:
{code}
    FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);

    int bOffset = 4;
    byte[] b10 = Bytes.toBytes(10); 
    Filter b10Filter = new RowFilter(CompareFilter.CompareOp.GREATER,
            new BinaryComponentComparator(b10,bOffset));
    filterList.addFilter(b10Filter);

    byte[] b20  = Bytes.toBytes(20);
    Filter b20Filter = new RowFilter(CompareFilter.CompareOp.LESS,
            new BinaryComponentComparator(b20,bOffset));
    filterList.addFilter(b20Filter);

    int cOffset = 8;
    byte[] c90  = Bytes.toBytes(90);
    Filter c90Filter = new RowFilter(CompareFilter.CompareOp.GREATER,
            new BinaryComponentComparator(c90,cOffset));
    filterList.addFilter(c90Filter);

    byte[] c100  = Bytes.toBytes(100);
    Filter c100Filter = new RowFilter(CompareFilter.CompareOp.LESS,
            new BinaryComponentComparator(c100,cOffset));
    filterList.addFilter(c100Filter);

    in dOffset = 12;
    byte[] d1   = Bytes.toBytes(1);
    Filter dFilter  = new RowFilter(CompareFilter.CompareOp.EQUAL,
            new BinaryComponentComparator(d1,dOffset));

    filterList.addFilter(dFilter);

    //build start and end key for scan
    int aOffset = 0;
    byte[] startKey = new byte[16]; //key size with four ints
    Bytes.putInt(startKey,aOffset,1); //a=1
    Bytes.putInt(startKey,bOffset,11); //b=11, takes care of b > 10
    Bytes.putInt(startKey,cOffset,91); //c=91, 
    Bytes.putInt(startKey,dOffset,1); //d=1, 

    byte[] endKey = new byte[16];
    Bytes.putInt(endKey,aOffset,1); //a=1
    Bytes.putInt(endKey,bOffset,20); //b=20, takes care of b < 20
    Bytes.putInt(endKey,cOffset,100); //c=100, 
    Bytes.putInt(endKey,dOffset,1); //d=1, 

    //setup scan
    Scan scan = new Scan(startKey,endKey);
    scan.setFilter(filterList);

    //The scanner below now should give only desired rows.
    //No client side filtering is required. 
    ResultScanner scanner = table.getScanner(scan);
{code}

The comparator can be used with any filter which makes use of 
ByteArrayComparable. Most notably it can be used with ValueFilter to filter out 
KV based on partial comparison of 'values' :
{code}
    byte[] partialValue = Bytes.toBytes("partial_value");
    int partialValueOffset = 
    Filter partialValueFilter = new ValueFilter(CompareFilter.CompareOp.GREATER,
            new BinaryComponentComparator(partialValue,partialValueOffset));
{code}

Which in turn can be combined with RowFilter to create a poweful predicate:
{code}
    RowFilter rowFilter = new RowFilter(GREATER, new 
BinaryComponentComparator(Bytes.toBytes("a"),1);
    FilterLiost fl = new FilterList 
(MUST_PASS_ALL,rowFilter,partialValueFilter);
{code}

  was:
Lets say you have composite key: a+b+c+d. And for simplicity assume that a,b,c, 
and d all are 4 byte integers.

Now, if you want to execute a query which is semantically same to following sql:

"SELECT * from table where a=1 and b > 10 and b < 20 and c > 90 and c < 100 and 
d=1"

The only choice you have is to do client side filtering. That could be lots of 
unwanted data going through various software components and network.

Solution:

We can create a "component" comparator which takes the value of the "component" 
and its relative position in the key to pass the 'Filter' subsystem of the 
server:

 

{{    FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);

    int bOffset = 4;
    byte[] b10 = Bytes.toBytes(10); 
    Filter b10Filter = new RowFilter(CompareFilter.CompareOp.GREATER,
            new BinaryComponentComparator(b10,bOffset));
    filterList.addFilter(b10Filter);

    byte[] b20  = Bytes.toBytes(20);
    Filter b20Filter = new RowFilter(CompareFilter.CompareOp.LESS,
            new BinaryComponentComparator(b20,bOffset));
    filterList.addFilter(b20Filter);

    int cOffset = 8;
    byte[] c90  = Bytes.toBytes(90);
    Filter c90Filter = new RowFilter(CompareFilter.CompareOp.GREATER,
            new BinaryComponentComparator(c90,cOffset));
    filterList.addFilter(c90Filter);

    byte[] c100  = Bytes.toBytes(100);
    Filter c100Filter = new RowFilter(CompareFilter.CompareOp.LESS,
            new BinaryComponentComparator(c100,cOffset));
    filterList.addFilter(c100Filter);

    in dOffset = 12;
    byte[] d1   = Bytes.toBytes(1);
    Filter dFilter  = new RowFilter(CompareFilter.CompareOp.EQUAL,
            new BinaryComponentComparator(d1,dOffset));

    filterList.addFilter(dFilter);

    //build start and end key for scan
    int aOffset = 0;
    byte[] startKey = new byte[16]; //key size with four ints
    Bytes.putInt(startKey,aOffset,1); //a=1
    Bytes.putInt(startKey,bOffset,11); //b=11, takes care of b > 10
    Bytes.putInt(startKey,cOffset,91); //c=91, 
    Bytes.putInt(startKey,dOffset,1); //d=1, 

    byte[] endKey = new byte[16];
    Bytes.putInt(endKey,aOffset,1); //a=1
    Bytes.putInt(endKey,bOffset,20); //b=20, takes care of b < 20
    Bytes.putInt(endKey,cOffset,100); //c=100, 
    Bytes.putInt(endKey,dOffset,1); //d=1, 

    //setup scan
    Scan scan = new Scan(startKey,endKey);
    scan.setFilter(filterList);

    //The scanner below now should give only desired rows.
    //No client side filtering is required. 
    ResultScanner scanner = table.getScanner(scan);}}

The comparator can be used with any filter which makes use of 
ByteArrayComparable. Most notably it can be used with ValueFilter to filter out 
KV based on partial comparison of 'values' :

 

{{    byte[] partialValue = Bytes.toBytes("partial_value");
    int partialValueOffset = 
    Filter partialValueFilter = new ValueFilter(CompareFilter.CompareOp.GREATER,
            new BinaryComponentComparator(partialValue,partialValueOffset));}}

Which in turn can be combined with RowFilter to create a poweful predicate:

 

{{    RowFilter rowFilter = new RowFilter(GREATER, new 
BinaryComponentComparator(Bytes.toBytes("a"),1);
    FilterLiost fl = new FilterList 
(MUST_PASS_ALL,rowFilter,partialValueFilter);}}


> A new binary component comparator(BinaryComponentComparator) to perform 
> comparison of arbitrary length and position
> -------------------------------------------------------------------------------------------------------------------
>
>                 Key: HBASE-22969
>                 URL: https://issues.apache.org/jira/browse/HBASE-22969
>             Project: HBase
>          Issue Type: Improvement
>          Components: Filters
>            Reporter: Udai Bhan Kashyap
>            Priority: Minor
>
> Lets say you have composite key: a+b+c+d. And for simplicity assume that 
> a,b,c, and d all are 4 byte integers.
> Now, if you want to execute a query which is semantically same to following 
> sql:
> {{"SELECT * from table where a=1 and b > 10 and b < 20 and c > 90 and c < 100 
> and d=1"}}
> The only choice you have is to do client side filtering. That could be lots 
> of unwanted data going through various software components and network.
> Solution:
> We can create a "component" comparator which takes the value of the 
> "component" and its relative position in the key to pass the 'Filter' 
> subsystem of the server:
> {code}
>     FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
>     int bOffset = 4;
>     byte[] b10 = Bytes.toBytes(10); 
>     Filter b10Filter = new RowFilter(CompareFilter.CompareOp.GREATER,
>             new BinaryComponentComparator(b10,bOffset));
>     filterList.addFilter(b10Filter);
>     byte[] b20  = Bytes.toBytes(20);
>     Filter b20Filter = new RowFilter(CompareFilter.CompareOp.LESS,
>             new BinaryComponentComparator(b20,bOffset));
>     filterList.addFilter(b20Filter);
>     int cOffset = 8;
>     byte[] c90  = Bytes.toBytes(90);
>     Filter c90Filter = new RowFilter(CompareFilter.CompareOp.GREATER,
>             new BinaryComponentComparator(c90,cOffset));
>     filterList.addFilter(c90Filter);
>     byte[] c100  = Bytes.toBytes(100);
>     Filter c100Filter = new RowFilter(CompareFilter.CompareOp.LESS,
>             new BinaryComponentComparator(c100,cOffset));
>     filterList.addFilter(c100Filter);
>     in dOffset = 12;
>     byte[] d1   = Bytes.toBytes(1);
>     Filter dFilter  = new RowFilter(CompareFilter.CompareOp.EQUAL,
>             new BinaryComponentComparator(d1,dOffset));
>     filterList.addFilter(dFilter);
>     //build start and end key for scan
>     int aOffset = 0;
>     byte[] startKey = new byte[16]; //key size with four ints
>     Bytes.putInt(startKey,aOffset,1); //a=1
>     Bytes.putInt(startKey,bOffset,11); //b=11, takes care of b > 10
>     Bytes.putInt(startKey,cOffset,91); //c=91, 
>     Bytes.putInt(startKey,dOffset,1); //d=1, 
>     byte[] endKey = new byte[16];
>     Bytes.putInt(endKey,aOffset,1); //a=1
>     Bytes.putInt(endKey,bOffset,20); //b=20, takes care of b < 20
>     Bytes.putInt(endKey,cOffset,100); //c=100, 
>     Bytes.putInt(endKey,dOffset,1); //d=1, 
>     //setup scan
>     Scan scan = new Scan(startKey,endKey);
>     scan.setFilter(filterList);
>     //The scanner below now should give only desired rows.
>     //No client side filtering is required. 
>     ResultScanner scanner = table.getScanner(scan);
> {code}
> The comparator can be used with any filter which makes use of 
> ByteArrayComparable. Most notably it can be used with ValueFilter to filter 
> out KV based on partial comparison of 'values' :
> {code}
>     byte[] partialValue = Bytes.toBytes("partial_value");
>     int partialValueOffset = 
>     Filter partialValueFilter = new 
> ValueFilter(CompareFilter.CompareOp.GREATER,
>             new BinaryComponentComparator(partialValue,partialValueOffset));
> {code}
> Which in turn can be combined with RowFilter to create a poweful predicate:
> {code}
>     RowFilter rowFilter = new RowFilter(GREATER, new 
> BinaryComponentComparator(Bytes.toBytes("a"),1);
>     FilterLiost fl = new FilterList 
> (MUST_PASS_ALL,rowFilter,partialValueFilter);
> {code}



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to