[
https://issues.apache.org/jira/browse/HBASE-10850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13957368#comment-13957368
]
Anoop Sam John commented on HBASE-10850:
----------------------------------------
I fear the optimization of essential CF is broken!
So this is not a case with SCVF filterIfMissing alone.
{code}
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()) {
results.clear();
boolean moreRows = nextRow(currentRow, offset, length);
if (!moreRows) return false;
// This row was totally filtered out, if this is NOT the last row,
// we should continue on. Otherwise, nothing else to do.
if (!stopRow) continue;
return false;
}
// Ok, we are done with storeHeap for this row.
// Now we may need to fetch additional, non-essential data into row.
// These values are not needed for filter to work, so we postpone
their
// fetch to (possibly) reduce amount of data loads from disk.
if (this.joinedHeap != null) {
KeyValue nextJoinedKv = joinedHeap.peek();
.....
...
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}
So here 1st we have scanned the essential CFs and filter is applied. isEmptyRow
is false initially but FilterWarpper#filterRowCells(List) cleared that results
list.
isEmptyRow boolean is false only and a call to filterRow() will return false
always.
Because of this we will continue with scan on the non essential CF. This is
against the optimization!!!
Also we added postScannerFilterRow CP hook in such a case where Filter is
filtering out a row. I fear this call also wont happen now as we are not going
inside the if (isEmptyRow || filterRow()) block.
> 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
> Priority: Critical
> Attachments: 10850-hasFilterRow-v1.txt, 10850-hasFilterRow-v2.txt,
> 10850-hasFilterRow-v3.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)