Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-22969

A new binary component comparator(BinaryComponentComparator) to perform comparison of arbitrary length and position

    XMLWordPrintableJSON

Details

    • New Feature
    • Status: Resolved
    • Minor
    • Resolution: Fixed
    • None
    • 3.0.0-alpha-1, 2.3.0, 2.2.3
    • Filters
    • None
    • Hide
      With BinaryComponentCompartor applications will be able to design diverse and powerful set of filters for rows and columns. See https://issues.apache.org/jira/browse/HBASE-22969 for example. In general, the comparator can be used with any filter taking ByteArrayComparable. As of now, following filters take ByteArrayComparable:

      1. RowFilter
      2. ValueFilter
      3. QualifierFilter
      4. FamilyFilter
      5. ColumnValueFilter
      Show
      With BinaryComponentCompartor applications will be able to design diverse and powerful set of filters for rows and columns. See https://issues.apache.org/jira/browse/HBASE-22969 for example. In general, the comparator can be used with any filter taking ByteArrayComparable. As of now, following filters take ByteArrayComparable: 1. RowFilter 2. ValueFilter 3. QualifierFilter 4. FamilyFilter 5. ColumnValueFilter

    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:

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

      Attachments

        1. HBASE-22969.0014.patch
          22 kB
          Udai Bhan Kashyap
        2. HBASE-22969.0013.patch
          22 kB
          Udai Bhan Kashyap
        3. HBASE-22969.0012.patch
          22 kB
          Udai Bhan Kashyap
        4. HBASE-22969.0011.patch
          19 kB
          Udai Bhan Kashyap
        5. HBASE-22969.0010.patch
          19 kB
          Udai Bhan Kashyap
        6. HBASE-22969.0009.patch
          9 kB
          Udai Bhan Kashyap
        7. HBASE-22969.0008.patch
          9 kB
          Udai Bhan Kashyap
        8. HBASE-22969.0007.patch
          8 kB
          Udai Bhan Kashyap
        9. HBASE-22969.0006.patch
          8 kB
          Udai Bhan Kashyap
        10. HBASE-22969.0005.patch
          8 kB
          Udai Bhan Kashyap
        11. HBASE-22969.0004.patch
          8 kB
          Udai Bhan Kashyap
        12. HBASE-22969.0003.patch
          8 kB
          Udai Bhan Kashyap
        13. HBASE-22969.HBASE-22969.0001.patch
          8 kB
          Udai Bhan Kashyap
        14. HBASE-22969.master.0001.patch
          8 kB
          Udai Bhan Kashyap

        Activity

          People

            udaikashyap Udai Bhan Kashyap
            udaikashyap Udai Bhan Kashyap
            Votes:
            1 Vote for this issue
            Watchers:
            18 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: