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

Vasu Mariyala commented on HBASE-8930:
--------------------------------------

Scanning for the results in a region is done in the following way (Note: I may 
have missed some of the conditions)

a) RegionScannerImpl has a store heap which contains key value scanner objects 
for a region like the storescanner (opens scanners across memstore, snapshot 
and all store files)

b) Based on few configurations, the scanner is seeked to the start key of the 
request.

c) For each row until end is reached or the batch size is reached, it does the 
following

   c.a) calls the filterRowKey on the filter object. If it returns true, it 
will go to the next row.

   c.b) If the above call returns false, it does the following.

       c.b.a) Set the row read in (c) as the current row

       c.b.b) While the next key value is not null

          c.b.b.a) Reads the next key value and ensures that is greater than 
the previous key value

          c.b.b.b) Calls the ScanQueryMatcher match method. It checks if the 
current key value belongs to the same row as the one read in step (c). It calls 
the filter logic and then checks if the keyValue corresponds to the column of 
interest and if the number of versions returned matches the ones specified 
using min and max versions.

          c.b.b.c) Based on the return type, it does the appropriate action. 
For include, it includes and goes to the next key value. If it is include and 
seek next column, the result is included and the scanner is asked to seek to 
the next column of interest. If it is include and seek next row, the result is 
included and the scanner is asked to seek to the next row.  If it is done, it 
returns. If it is seek next row, the scanner is asked to seek to the next row. 
If it is seek next column, the scanner is asked to seek to the next column of 
interest.


To illustrate the failing scenario mentioned in example 

a) Once key value in (c.b.b.b) is for the column 5594 and the appropriate 
number of versions for the column are read, it returns include and seek next 
column.

b) in c.b.b.c, the scanner is asked to the next column of interest which is 
5596. But since there is no value for that column, the scanner seeks to the 
column 5601 (the one after 5596)

c) The loop of c.b.b repeats itself and since the filtering is done before the 
actual check of what columns are requested for, the filterKeyValue method is 
invoked.
                
> Filter evaluates KVs outside requested columns
> ----------------------------------------------
>
>                 Key: HBASE-8930
>                 URL: https://issues.apache.org/jira/browse/HBASE-8930
>             Project: HBase
>          Issue Type: Bug
>          Components: Filters
>    Affects Versions: 0.94.7
>            Reporter: Federico Gaule
>            Assignee: Vasu Mariyala
>            Priority: Critical
>              Labels: filters, hbase, keyvalue
>         Attachments: HBASE-8930.patch
>
>
> 1- Fill row with some columns
> 2- Get row with some columns less than universe - Use filter to print kvs
> 3- Filter prints not requested columns
> Filter (AllwaysNextColFilter) always return ReturnCode.INCLUDE_AND_NEXT_COL 
> and prints KV's qualifier
> SUFFIX_0 = 0
> SUFFIX_1 = 1
> SUFFIX_4 = 4
> SUFFIX_6 = 6
> P= Persisted
> R= Requested
> E= Evaluated
> X= Returned
> | 5580 | 5581 | 5584 | 5586 | 5590 | 5591 | 5594 | 5596 | 5600 | 5601 | 5604 
> | 5606 |... 
> |      |  P   |   P  |      |      |  P   |   P  |      |      |  P   |   P  
> |      |...
> |      |  R   |   R  |   R  |      |  R   |   R  |   R  |      |      |      
> |      |...
> |      |  E   |   E  |      |      |  E   |   E  |      |      |  
> {color:red}E{color}   |      |      |...
> |      |  X   |   X  |      |      |  X   |   X  |      |      |      |      
> |      |
> {code:title=ExtraColumnTest.java|borderStyle=solid}
>     @Test
>     public void testFilter() throws Exception {
>         Configuration config = HBaseConfiguration.create();
>         config.set("hbase.zookeeper.quorum", "myZK");
>         HTable hTable = new HTable(config, "testTable");
>         byte[] cf = Bytes.toBytes("cf");
>         byte[] row = Bytes.toBytes("row");
>         byte[] col1 = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 558, (byte) SUFFIX_1));
>         byte[] col2 = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 559, (byte) SUFFIX_1));
>         byte[] col3 = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 560, (byte) SUFFIX_1));
>         byte[] col4 = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 561, (byte) SUFFIX_1));
>         byte[] col5 = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 562, (byte) SUFFIX_1));
>         byte[] col6 = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 563, (byte) SUFFIX_1));
>         byte[] col1g = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 558, (byte) SUFFIX_6));
>         byte[] col2g = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 559, (byte) SUFFIX_6));
>         byte[] col1v = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 558, (byte) SUFFIX_4));
>         byte[] col2v = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 559, (byte) SUFFIX_4));
>         byte[] col3v = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 560, (byte) SUFFIX_4));
>         byte[] col4v = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 561, (byte) SUFFIX_4));
>         byte[] col5v = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 562, (byte) SUFFIX_4));
>         byte[] col6v = new QualifierConverter().objectToByteArray(new 
> Qualifier((short) 563, (byte) SUFFIX_4));
>         // =========== INSERTION =============//
>         Put put = new Put(row);
>         put.add(cf, col1, Bytes.toBytes((short) 1));
>         put.add(cf, col2, Bytes.toBytes((short) 1));
>         put.add(cf, col3, Bytes.toBytes((short) 3));
>         put.add(cf, col4, Bytes.toBytes((short) 3));
>         put.add(cf, col5, Bytes.toBytes((short) 3));
>         put.add(cf, col6, Bytes.toBytes((short) 3));
>         hTable.put(put);
>         put = new Put(row);
>         put.add(cf, col1v, Bytes.toBytes((short) 10));
>         put.add(cf, col2v, Bytes.toBytes((short) 10));
>         put.add(cf, col3v, Bytes.toBytes((short) 10));
>         put.add(cf, col4v, Bytes.toBytes((short) 10));
>         put.add(cf, col5v, Bytes.toBytes((short) 10));
>         put.add(cf, col6v, Bytes.toBytes((short) 10));
>         hTable.put(put);
>         hTable.flushCommits();
>         //==============READING=================//
>         Filter allwaysNextColFilter = new AllwaysNextColFilter();
>         Get get = new Get(row);
>         get.addColumn(cf, col1); //5581
>         get.addColumn(cf, col1v); //5584
>         get.addColumn(cf, col1g); //5586
>         get.addColumn(cf, col2); //5591
>         get.addColumn(cf, col2v); //5594        
>         get.addColumn(cf, col2g); //5596
>         
>         get.setFilter(allwaysNextColFilter);
>         get.setMaxVersions(1);
>         System.out.println(get);
>         Scan scan = new Scan(get);
>         ResultScanner scanner = hTable.getScanner(scan);
>         Iterator<Result> iterator = scanner.iterator();
>         System.out.println("SCAN");
>         while (iterator.hasNext()) {
>             Result next = iterator.next();
>             for (KeyValue kv : next.list()) {
>                 System.out.println(new 
> QualifierConverter().byteArrayToObject(kv.getQualifier()));
>             }
>         }
>     }
> }
> {code}
> Requested 5581 5584 5586 5591 5594 5596
> NOT REQUESTED: 5561
> Sysout Filter
> {noformat}
> \x00\x00\x1A\xBE\x00\x05^:\x00\x00\xA0X\x00\x00=\x1A/H0:\x02.\x01/1373577819267/Put/vlen=2/ts=2
> Qualifier{date=558, type=SUFFIX_1}
> \x00\x00\x1A\xBE\x00\x05^:\x00\x00\xA0X\x00\x00=\x1A/H0:\x02.\x02/1373577819272/Put/vlen=2/ts=3
> Qualifier{date=558, type=SUFFIX_4}
> \x00\x00\x1A\xBE\x00\x05^:\x00\x00\xA0X\x00\x00=\x1A/H0:\x02/\x01/1373577819267/Put/vlen=2/ts=2
> ualifier{date=559, type=SUFFIX_1}
> \x00\x00\x1A\xBE\x00\x05^:\x00\x00\xA0X\x00\x00=\x1A/H0:\x02/\x02/1373577819272/Put/vlen=2/ts=3
> Qualifier{date=559, type=SUFFIX_4}
>  
> \x00\x00\x1A\xBE\x00\x05^:\x00\x00\xA0X\x00\x00=\x1A/H0:\x020\x01/1373577819267/Put/vlen=2/ts=2
> Qualifier{date=560, type=SUFFIX_1} (DATE 5601 NOT REQUESTED BUT EVALUATED)
> {noformat}
> Sysout ExtraColumnTest
> {noformat}
> {"timeRange":[0,9223372036854775807],"totalColumns":6,"cacheBlocks":true,"families":{"H0":["\\x02.\\x01","\\x02.\\x02","\\x02.\\x06","\\x02/\\x01"]},"maxVersions":1,"filter":"AllwaysNextColFilter","row":"\\x00\\x00\\x1A\\xBE\\x00\\x05^:\\x00\\x00\\xA0X\\x00\\x00=\\x1A"}
> SCAN
> Qualifier{date=558, type=SUFFIX_1}
> Qualifier{date=558, type=SUFFIX_4}
> Qualifier{date=559, type=SUFFIX_1}
> Qualifier{date=559, type=SUFFIX_4}
> {noformat}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to