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 6AF1EDD03 for ; Tue, 21 May 2013 07:49:20 +0000 (UTC) Received: (qmail 43421 invoked by uid 500); 21 May 2013 07:49:19 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 43366 invoked by uid 500); 21 May 2013 07:49:19 -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 43298 invoked by uid 99); 21 May 2013 07:49:18 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 21 May 2013 07:49:18 +0000 Date: Tue, 21 May 2013 07:49:18 +0000 (UTC) From: "Liang Xie (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HBASE-8555) FilterList correctness was dominated by sub-filter(list) ordering randomly 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-8555?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13662783#comment-13662783 ] Liang Xie commented on HBASE-8555: ---------------------------------- bq. How about call the Rowfilter#filterRowKey() in the Rowfilter#filterKeyValue() if it haven't been called seems it's not a perfect solution for me, since the filter list maybe include other filters, not just row filter:) probably we could improve it in Filter&FilterBase. [~lhofhansl], would you mind having an eye on this jira, thanks. > FilterList correctness was dominated by sub-filter(list) ordering randomly > -------------------------------------------------------------------------- > > Key: HBASE-8555 > URL: https://issues.apache.org/jira/browse/HBASE-8555 > Project: HBase > Issue Type: Bug > Components: Filters > Affects Versions: 0.94.3 > Reporter: Liang Xie > Assignee: Liang Xie > Priority: Critical > Attachments: 8555-trunk-v1.txt, HBASE-8555-0.94.txt, HBASE-8555-0.94-v2.txt > > > say, ther're 10 rows, column value is i%2: > row0 0 > row1 1 > row2 0 > row3 1 > row4 0 > row5 1 > row6 0 > row7 1 > row8 0 > row9 1 > 1: filter : row filter > row4 ===> row5 row6 row7 row8 row9 > 2: subFilterList: row filter <= row4 && column==0 ===> row0 row2 row4 > 3.1 filterlist[expected] filter || subFilterList ===> row0 row2 row4 row5 row6 row7 row8 row9 > 3.2 filterlist[BUGON!] subFilterList || filter ===> row0 row1 row2 row3 row4 row5 row6 row7 row8 row9 > (Please refer to the new testNestedFilterListWithSCVF case) > It was found when i managed to transform the following SQL into HBase scan statement: > select xxx from xxx where (pk <= xxx and column1 = xxx) or pk > xxx > My finding is that we had an assumption for filter methods call sequence: > e.g. filterRowKey() should be called before filterKeyValue(). > and the orignial filterList.filterRowKey impl broke it due to fast short-circuit returning. -- 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