[
https://issues.apache.org/jira/browse/HBASE-10850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13950337#comment-13950337
]
Anoop Sam John commented on HBASE-10850:
----------------------------------------
{code}
HRegion#nextInternal(List<Cell> results, int limit)
// save that the row was empty before filters applied to it.
final boolean isEmptyRow = results.isEmpty();
// We have the part of the row necessary for filtering (all of it, usually).
// First filter with the filterRow(List).
if (filter != null && filter.hasFilterRow()) {
filter.filterRowCells(results);
}
if (isEmptyRow || filterRow()) {
...
private boolean filterRow() throws IOException {
// when hasFilterRow returns true, filter.filterRow() will be called
automatically inside
// filterRowCells(List<Cell> kvs) so we skip that scenario here.
return filter != null && (!filter.hasFilterRow())
&& filter.filterRow();
}
{code}
In 96+ version filterRowCells(List) is internally calling filterRow() also and
if that return true, just clears the passed cells list. (SCVF uses filterRow())
And u can see private boolean filterRow() wont get executed because of this
(!filter.hasFilterRow()).
results is no empty before applying the 1st filter op.
So here we have to make change as
{code}
if (results.isEmpty() || filterRow()) {
....
}
{code}
Pls correct if I am wrong..
> 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: haosdent
> Attachments: HBASE-10850-96.patch, HBASE-10850.patch,
> HBaseSingleColumnValueFilterTest.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)