hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Anoop Sam John (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-10850) essential column family optimization is broken
Date Thu, 03 Apr 2014 09:41:15 GMT

    [ https://issues.apache.org/jira/browse/HBASE-10850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13958668#comment-13958668
] 

Anoop Sam John commented on HBASE-10850:
----------------------------------------

Essential family optimization is broken when Scan selects column(s) other than the condition
column from essential family.  When only the conditioned column is selected the optimization
still works.

> 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.98.1, 0.99.0, 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)

Mime
View raw message