[ https://issues.apache.org/jira/browse/HBASE-17375?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15778096#comment-15778096 ]
Chang chen edited comment on HBASE-17375 at 12/26/16 10:36 AM: --------------------------------------------------------------- I already wrote a test case in a standalone application, I will upload a UT patch ASAP was (Author: baibaichen): I already write a test case in a standalone application, I will apply a UT patch ASAP > PrefixTreeArrayReversibleScanner#previousRowInternal doesn't work correctly > --------------------------------------------------------------------------- > > Key: HBASE-17375 > URL: https://issues.apache.org/jira/browse/HBASE-17375 > Project: HBase > Issue Type: Bug > Affects Versions: 1.1.7, 0.98.23, 0.98.24 > Reporter: Chang chen > Attachments: row trie example.PNG > > > Recently, we find our hbase compaction thread never end. Assume we have > following cells: > {quote} > <A,a> 1 > <A,v> 1 > <Aaeeee,a> 1 > <Aaeeee,v> 1 > <Abc,a> 1 > <Abc,v> 1 > <Abde,a> 1 > <Abde,v> 1 > {quote} > If we encode above datas into prefix tree block, then it looks like: > !row trie example.PNG! > Assume the current row is {color:red}Abc{color} (e.g. the current row node is > 4), then the previous row should be *Aaeeee* (e.g. 2). However > previousRowInternal return {color:red}A{color}(e.g. 1) > After investigation, I believe it's the bug of > PrefixTreeArrayReversibleScanner#previousRowInternal. > {code} > private boolean previousRowInternal() { > //... > while (!beforeFirst) { > //.... > // what if currentRowNode is nub? > if (currentRowNode.hasOccurrences()) {// escape clause > currentRowNode.resetFanIndex(); > return true;// found some values > } > } > {code} > currentRowNode.hasOccurrences() only test whether it has cell or not. But in > the case of currentRowNode.isNub() is true, previousRowInternal should > follow the previous fan instead of return. -- This message was sent by Atlassian JIRA (v6.3.4#6332)