Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 41477200BFB for ; Tue, 27 Dec 2016 09:34:00 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 40129160B23; Tue, 27 Dec 2016 08:34:00 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 8CB0A160B31 for ; Tue, 27 Dec 2016 09:33:59 +0100 (CET) Received: (qmail 46893 invoked by uid 500); 27 Dec 2016 08:33:58 -0000 Mailing-List: contact issues-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@hbase.apache.org Received: (qmail 46864 invoked by uid 99); 27 Dec 2016 08:33:58 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 27 Dec 2016 08:33:58 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 70E4A2C2A66 for ; Tue, 27 Dec 2016 08:33:58 +0000 (UTC) Date: Tue, 27 Dec 2016 08:33:58 +0000 (UTC) From: "Chang chen (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (HBASE-17375) PrefixTreeArrayReversibleScanner#previousRowInternal doesn't work correctly MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Tue, 27 Dec 2016 08:34:00 -0000 [ https://issues.apache.org/jira/browse/HBASE-17375?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chang chen updated HBASE-17375: ------------------------------- Attachment: (was: HBASE_17375_v1.patch) > 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} > 1 > 1 > 1 > 1 > 1 > 1 > 1 > 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)