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

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

Sorry Ted.
I am not worried abt filterIfMissing here at all..  Just consider there is no 
case of missing the conditioned column at all..
I am saying wrt the V4 patch
There we are treating filterRow() returning true   and filterRowCells(List) 
clearing the list as same.   Correct (?)
What I am saying is this is wrong.
As u can see in my above example when a row is having  F1:Q1  with value as V1 
we have to include this row. SingleColumnValueExcludeFilter uses filterRow() 
method from SingleColumnValueFilter  and it will return false in that case.  
You can see SingleColumnValueExcludeFilter  is having filterRowCells(List) 
also.  This will remove the KVs of column F1:Q1.  So once we call this  the 
results list will be empty.  As per patch V4 it will be considered like the row 
is being filtered out and we will go with next row..  But this is wrong.  In 
this case we should have been going with non essential family data scan and 
include those KVs in the result..  Hope I am making it clear.

Again  the bug is found with SCVF filterIfMissing case.  But this is a more 
serious issue.  Thanks for finding out this.  
-Now the essential CF optimization is broken. 
- Also a new hook added when filter is filtering out a row. This new hook is 
also not getting called now.  Huawei's HIndex solution, scan perf gain comes 
from this hook only. 

> 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)

Reply via email to