Return-Path: X-Original-To: apmail-hbase-issues-archive@www.apache.org Delivered-To: apmail-hbase-issues-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 6BB48116CF for ; Thu, 14 Aug 2014 10:10:13 +0000 (UTC) Received: (qmail 49847 invoked by uid 500); 14 Aug 2014 10:10:13 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 49805 invoked by uid 500); 14 Aug 2014 10:10:13 -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 49793 invoked by uid 99); 14 Aug 2014 10:10:13 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 14 Aug 2014 10:10:13 +0000 Date: Thu, 14 Aug 2014 10:10:13 +0000 (UTC) From: "ramkrishna.s.vasudevan (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HBASE-11728) Some data miss when scan using PREFIX_TREE DATA-BLOCK-ENCODING MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HBASE-11728?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14096819#comment-14096819 ] ramkrishna.s.vasudevan commented on HBASE-11728: ------------------------------------------------ In other encoding algos, while seeking using BufferedDataEncoder we do moveToPrevious() by just ensuring that the data current kv starting offset is stored in the seeker so that when the actual next() is called we could start from that same position. We should indicate in the PrefixTreeSeeker that whenever we do a next() after seek() and we know that the seek() has done a previous() we should make the advance() to follow the current fan out even though there could be other fanouts. > 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)