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) Unexpected behavior when using filter SingleColumnValueFilter
Date Wed, 02 Apr 2014 18:30:18 GMT

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

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

bq.Patch v4 would still allow this hook to be called.
Agree. It is calling
bq.I would say if SingleColumnValueExcludeFilter#filterRowCells(List) removes the KeyValue,
leading to empty List, but SingleColumnValueExcludeFilter#filterRow() returns false, that
is inconsistent.
Sorry I have to disagree.
As per my above scan example
{code}
Scan s = new Scan();
s.addColumn(F1, Q1);
s.addFamily(F2);
SingleColumnValueExcludeFilter f = new SingleColumnValueExcludeFilter(F1, Q1, CompareOp.EQUAL,
V1);
s.setFilter(f);
{code}
Say for row r1 F1:Q1 = V1.   We select only this column from cf F1.  So the results will contain
one KV in it. Now when calling filterRowCells(List) in SCVEF this will remove the cell.  We
have to exclude the conditioned column from result.    But the condition was passing for this
row.  So filterRow() must return false.     This is very much acceptable.    
Note : Here the filterRowCells() called not all all the column kvs.  Only we fetch essential
CF 1st. 


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

Mime
View raw message