[
https://issues.apache.org/jira/browse/HBASE-20151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16554084#comment-16554084
]
Zheng Hu commented on HBASE-20151:
----------------------------------
Reconsider the bug again, I think the cause is not the NEXT_ROW of
FamilyFilter, the current code process as following:
||input|| FilterList || SVCF || FF || comment||
|a:1| NEXT_ROW| INCLUDE | NEXT_ROW | will swtich to next family |
|a:10| skip to test| skip to test| skip to test| |
|b:2|INCLUDE | INCLUDE | INCLUDE| |
In fact, the FilterList worked as expected, but the SVCF#filterRow() will
return true for the entire row cells, because
{code}
@Override
public boolean filterRow() {
// If column was found, return false if it was matched, true if it was not
// If column not found, return true if we filter if missing, false if not
return this.foundColumn? !this.matchedColumn: this.filterIfMissing; -->
skip to test a:10, so foundColumn is false, and we set filterIfMissing to
true... so return true finally...
}
{code}
In the end, the RegionScannerImpl#nextInternal will filter out the entire
row....
https://github.com/apache/hbase/blob/master/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java#L6665
Let's consider the generic case, filterList = SVCF AND filter1 (the filter1
has not to be FamilyFilter), if filter1 return NEXT_ROW, then the bug will
still occur... So we try to fix the bug by changing FamilyFilter or FilterList,
it is in vain, I have to say....... :-)
I think we should reconsider the SVCF implementation. How do you guys think ?
> Bug with SingleColumnValueFilter and FamilyFilter
> -------------------------------------------------
>
> Key: HBASE-20151
> URL: https://issues.apache.org/jira/browse/HBASE-20151
> Project: HBase
> Issue Type: Bug
> Affects Versions: 2.1.0, 2.0.1, 1.4.5
> Environment: MacOS 10.13.3
> HBase 1.3.1
> Reporter: Steven Sadowski
> Assignee: Reid Chan
> Priority: Major
> Fix For: 3.0.0, 1.5.0, 2.2.0
>
> Attachments: HBASE-20151.master.001.patch,
> HBASE-20151.master.002.patch, HBASE-20151.master.003.patch,
> HBASE-20151.master.004.patch, HBASE-20151.master.004.patch,
> HBASE-20151.master.005.patch, HBASE-20151.master.006.patch,
> filter-list-type.v1.txt
>
>
> When running the following queries, the result is sometimes return correctly
> and other times incorrectly based on the qualifier queried.
> Setup:
> {code:java}
> create 'test', 'a', 'b'
> test = get_table 'test'
> test.put '1', 'a:1', nil
> test.put '1', 'a:10', nil
> test.put '1', 'b:2', nil
> {code}
>
> This query works fine when the SCVF's qualifier has length 1 (i.e. '1') :
> {code:java}
> test.scan({ FILTER => "(
> SingleColumnValueFilter('a','1',=,'binary:',true,true) AND
> FamilyFilter(=,'binary:b') )"})
> ROW COLUMN+CELL
> 1 column=b:2,
> timestamp=1520455888059, value=
> 1 row(s) in 0.0060 seconds
> {code}
>
> The query should return the same result when passed a qualifier of length 2
> (i.e. '10') :
> {code:java}
> test.scan({ FILTER => "(
> SingleColumnValueFilter('a','10',=,'binary:',true,true) AND
> FamilyFilter(=,'binary:b') )"})
> ROW COLUMN+CELL
> 0 row(s) in 0.0110 seconds
> {code}
> However, in this case, it does not return any row (expected result would be
> to return the same result as the first query).
>
> Removing the family filter while the qualifier is '10' yields expected
> results:
> {code:java}
> test.scan({ FILTER => "(
> SingleColumnValueFilter('a','10',=,'binary:',true,true) )"})
> ROW COLUMN+CELL
> 1 column=a:1,
> timestamp=1520455887954, value=
> 1 column=a:10,
> timestamp=1520455888024, value=
> 1 column=b:2,
> timestamp=1520455888059, value=
> 1 row(s) in 0.0140 seconds
> {code}
>
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)