[
https://issues.apache.org/jira/browse/HBASE-10850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13957918#comment-13957918
]
Anoop Sam John commented on HBASE-10850:
----------------------------------------
I was also initially thinking that we can move the results empty check down
after calling the filterRowCells()
https://issues.apache.org/jira/browse/HBASE-10850?focusedCommentId=13950337&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13950337
But later relized there is an issue.
bq.When results is cleared by a call to filterRowCells(List) then also we will
treat it as a filter row. Is that correct?
Actually this is not correct. Consider the below Scan with Filter
{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}
Here F1 is essential CF and F2 is not. From F1 we scan only Q1 column and we
have a SingleColumnValueExcludeFilter on this.
Now for a row for which the condition is satisfied we will get the List
'results' with one entry before calling filterRowCells(List). But once we call
this the results will get cleared by the filterRowCells in SCVEF. This doesn't
mean that we have to filter out this row. On calling the filterRow() it is
going to return false. So in this case we must go ahead and scan the non
essential CF data also and include those in the returned result. But of we
treat filterRowCells(List) clearing the list as row being filtered out, we wont
get this behaviour.
I actually again forgot this point when I asked the Q on patch V4.Now only I
remembered it. :)
> 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)