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

ramkrishna.s.vasudevan commented on HBASE-11728:
------------------------------------------------

bq.So from the above analysis it shows that while doing previous() the fanIndex 
gets changed for the row node and hence while doing the actual next() the 
traversal of the trie structure ends up in the other leaf node attached to it.
I would like to rephrase the above sentence.  The issue is clear now.
During the seek phase 
{code}
[fan:0AB,token:a-b-,numCells:0,fanIndex:1(A), 
fan:-,token:-1,numCells:1,fanIndex:0(-), 
fan:29,token:14023,numCells:0,fanIndex:0(2), 
fan:,token:9600-1402396277,numCells:1,fanIndex:-1]
{code}
and we have ended up in the right kv.
Now the previous() has now made the current row to be pointing to 
{code}
fan:-,token:-1,numCells:1,fanIndex:0(-), 
{code}
Which is correct. 
Now on the next() call, ideally we should be traversing the same set of row 
nodes but as the node 
{code}
a-b-,numCells:0,fanIndex:1(A)
{code}
has more fanouts other than A (i.e B) the next() tries to track the nodes that 
are attached to B and thus we end up in altogether new row.  My feel is that in 
this scenario where we have already formed the set of nodes we should traverse 
the same set of nodes that we already created while seeking.
[~mcorgan] - What do you think?

> Some data miss when scan using PREFIX_TREE DATA-BLOCK-ENCODING
> --------------------------------------------------------------
>
>                 Key: HBASE-11728
>                 URL: https://issues.apache.org/jira/browse/HBASE-11728
>             Project: HBase
>          Issue Type: Bug
>    Affects Versions: 0.96.1.1, 0.98.4
>         Environment: ubuntu12 
> hadoop-2.2.0
> Hbase-0.96.1.1
> SUN-JDK(1.7.0_06-b24)
>            Reporter: wuchengzhi
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Critical
>             Fix For: 0.99.0, 2.0.0, 0.98.6
>
>         Attachments: 29cb562fad564b468ea9d61a2d60e8b0, HFileAnalys.java, 
> TestPrefixTree.java
>
>   Original Estimate: 72h
>  Remaining Estimate: 72h
>
> In Scan case, i prepare some data as beflow:
> Table Desc (Using the prefix-tree encoding) :
> 'prefix_tree_test', {NAME => 'cf_1', DATA_BLOCK_ENCODING => 'PREFIX_TREE', 
> TTL => '15552000'}
> and i put 5 rows as:
> (RowKey , Qualifier, Value)
> 'a-b-0-0', 'qf_1', 'c1-value'
> 'a-b-A-1', 'qf_1', 'c1-value'
> 'a-b-A-1-1402329600-1402396277', 'qf_2', 'c2-value'
> 'a-b-A-1-1402397227-1402415999', 'qf_2', 'c2-value-2'
> 'a-b-B-2-1402397300-1402416535', 'qf_2', 'c2-value-3'
> so i try to scan the rowKey between 'a-b-A-1' and 'a-b-A-1:' , i and got the 
> corret result:
> Test 1: 
> Scan scan = new Scan();
> scan.setStartRow("a-b-A-1".getBytes());
> scan.setStopRow("a-b-A-1:".getBytes());
> ------------------------------------------------------
> 'a-b-A-1', 'qf_1', 'c1-value'
> 'a-b-A-1-1402329600-1402396277', 'qf_2', 'c2-value'
> 'a-b-A-1-1402397227-1402415999', 'qf_2', 'c2-value-2'
> and then i try next , scan to addColumn
> Test2:
> Scan scan = new Scan();
> scan.addColumn(Bytes.toBytes("cf_1") ,  Bytes.toBytes("qf_2"));
> scan.setStartRow("a-b-A-1".getBytes());
> scan.setStopRow("a-b-A-1:".getBytes());
> ----------------------------------------------
> except:
> 'a-b-A-1-1402329600-1402396277', 'qf_2', 'c2-value'
> 'a-b-A-1-1402397227-1402415999', 'qf_2', 'c2-value-2'
> but actually i got nonthing. Then i update the addColumn for 
> scan.addColumn(Bytes.toBytes("cf_1") ,  Bytes.toBytes("qf_1")); and i got the 
> expected result 'a-b-A-1', 'qf_1', 'c1-value' as well.
> then i do more testing...  i update the case to modify the startRow greater 
> than the 'a-b-A-1' 
> Test3:
> Scan scan = new Scan();
> scan.setStartRow("a-b-A-1-".getBytes());
> scan.setStopRow("a-b-A-1:".getBytes());
> ------------------------------------------------------
> except:
> 'a-b-A-1-1402329600-1402396277', 'qf_2', 'c2-value'
> 'a-b-A-1-1402397227-1402415999', 'qf_2', 'c2-value-2'
> but actually  i got nothing again. i modify the start row greater than 
> 'a-b-A-1-1402329600-1402396277'
> Scan scan = new Scan();
> scan.setStartRow("a-b-A-1-140239".getBytes());
> scan.setStopRow("a-b-A-1:".getBytes());
> and i got the expect row as well:
> 'a-b-A-1-1402397227-1402415999', 'qf_2', 'c2-value-2'
> So, i think it may be a bug in the prefix-tree encoding.It happens after the 
> data flush to the storefile, and it's ok when the data in mem-store.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to