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 41D941001F for ; Fri, 4 Apr 2014 13:07:58 +0000 (UTC) Received: (qmail 4765 invoked by uid 500); 4 Apr 2014 13:07:35 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 4594 invoked by uid 500); 4 Apr 2014 13:07:33 -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 4523 invoked by uid 99); 4 Apr 2014 13:07:32 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 04 Apr 2014 13:07:32 +0000 Date: Fri, 4 Apr 2014 13:07:32 +0000 (UTC) From: "Hudson (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HBASE-10850) essential column family optimization is broken 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:comment-tabpanel&focusedCommentId=13959923#comment-13959923 ] Hudson commented on HBASE-10850: -------------------------------- SUCCESS: Integrated in hbase-0.96-hadoop2 #259 (See [https://builds.apache.org/job/hbase-0.96-hadoop2/259/]) HBASE-10850 essential column family optimization is broken (tedyu: rev 1584357) * /hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java * /hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java * /hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java > essential column family optimization is broken > ---------------------------------------------- > > Key: HBASE-10850 > URL: https://issues.apache.org/jira/browse/HBASE-10850 > Project: HBase > Issue Type: Bug > Components: Coprocessors, Filters, Performance > Affects Versions: 0.96.1.1 > Reporter: Fabien Le Gallo > Assignee: Ted Yu > Priority: Blocker > Fix For: 0.99.0, 0.98.2, 0.96.3 > > Attachments: 10850-hasFilterRow-v1.txt, 10850-hasFilterRow-v2.txt, 10850-hasFilterRow-v3.txt, 10850-v4.txt, 10850-v5.txt, 10850-v6.txt, 10850-v7.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. > +The analysis of this issue lead us to 2 critical bugs induced in 96 and above versions+ > 1. The essential family optimization is broken in some cases. In case of condition on some families, we 1st will read those KVs and apply condition on those, when the condition says to filter out that row, we will not go ahead and fetch data from remaining non essential CFs. But now in most of the cases we will do this unwanted data read which is fully against this optimization > 2. We have a CP hook postFilterRow() which will be called when a row is getting filtered out by the Filter. This gives the CP to do a reseek to the next known row which it thinks can evaluate the condition to true. But currently in 96+ code , this hook is not getting called. -- This message was sent by Atlassian JIRA (v6.2#6252)