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 0705210C18 for ; Wed, 2 Apr 2014 15:59:36 +0000 (UTC) Received: (qmail 55579 invoked by uid 500); 2 Apr 2014 15:59:33 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 55556 invoked by uid 500); 2 Apr 2014 15:59:32 -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 55464 invoked by uid 99); 2 Apr 2014 15:59:27 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 02 Apr 2014 15:59:27 +0000 Date: Wed, 2 Apr 2014 15:59:27 +0000 (UTC) From: "Ted Yu (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (HBASE-10850) Unexpected behavior when using filter SingleColumnValueFilter 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-10850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Yu updated HBASE-10850: --------------------------- Attachment: 10850-v4.txt Patch v4 is much simpler. Following tests passed: mt -Dtest=*Filter* Fabien's test also passed. > Unexpected behavior when using filter SingleColumnValueFilter > ------------------------------------------------------------- > > Key: HBASE-10850 > URL: https://issues.apache.org/jira/browse/HBASE-10850 > Project: HBase > Issue Type: Bug > Components: Filters > Affects Versions: 0.96.1.1 > Reporter: Fabien Le Gallo > Assignee: Ted Yu > Priority: Critical > Attachments: 10850-hasFilterRow-v1.txt, 10850-hasFilterRow-v2.txt, 10850-hasFilterRow-v3.txt, 10850-v4.txt, HBASE-10850-96.patch, HBASE-10850.patch, HBASE-10850_V2.patch, HBaseSingleColumnValueFilterTest.java, TestWithMiniCluster.java > > > When using the filter SingleColumnValueFilter, and depending of the columns specified in the scan (filtering column always specified), the results can be different. > Here is an example. > Suppose the following table: > ||key||a:foo||a:bar||b:foo||b:bar|| > |1|false|_flag_|_flag_|_flag_| > |2|true|_flag_|_flag_|_flag_| > |3| |_flag_|_flag_|_flag_| > With this filter: > {code} > SingleColumnValueFilter filter = new SingleColumnValueFilter(Bytes.toBytes("a"), Bytes.toBytes("foo"), CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("false"))); > filter.setFilterIfMissing(true); > {code} > Depending of how I specify the list of columns to add in the scan, the result is different. Yet, all examples below should always return only the first row (key '1'): > OK: > {code} > scan.addFamily(Bytes.toBytes("a")); > {code} > KO (2 results returned, row '3' without 'a:foo' qualifier is returned): > {code} > scan.addFamily(Bytes.toBytes("a")); > scan.addFamily(Bytes.toBytes("b")); > {code} > KO (2 results returned, row '3' without 'a:foo' qualifier is returned): > {code} > scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("foo")); > scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("bar")); > scan.addColumn(Bytes.toBytes("b"), Bytes.toBytes("foo")); > {code} > OK: > {code} > scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("foo")); > scan.addColumn(Bytes.toBytes("b"), Bytes.toBytes("bar")); > {code} > OK: > {code} > scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("foo")); > scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("bar")); > {code} > This is a regression as it was working properly on HBase 0.92. > You will find in attachement the unit tests reproducing the issue. -- This message was sent by Atlassian JIRA (v6.2#6252)