You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org> on 2011/12/29 23:40:31 UTC

[jira] [Commented] (HBASE-5104) FilterList doesn't work right with ColumnPaginationFilter

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

Lars Hofhansl commented on HBASE-5104:
--------------------------------------

@Kannan: Right, that's what I had in mind. You build up your filter (using whatever FilterList nesting you need) and then wrap the end result into a ColumnPaginationFilter (which would optionally take a filter). That way it can be applied at the end-stage while not requiring any API changes to Get/Scan.

I see what your saying, though, it rarely makes sense to wrap a ColumnPaginationFilter into a FilterList... We could just document it as such.

                
> FilterList doesn't work right with ColumnPaginationFilter
> ---------------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: testFilterList.rb
>
>
> Thanks Jiakai Liu for reporting this issue and doing the initial investigation. Email from Jiakai below:
> Assuming that we have an index column family with the following entries:
> "tag0:001:thread1"
> ...
> "tag1:001:thread1"
> "tag1:002:thread2"
> ...
> "tag1:010:thread10"
> ...
> "tag2:001:thread1"
> "tag2:005:thread5"
> ...
> To get threads with "tag1" in range [5, 10), I tried the following code:
>     ColumnPrefixFilter filter1 = new ColumnPrefixFilter(Bytes.toBytes("tag1"));
>     ColumnPaginationFilter filter2 = new ColumnPaginationFilter(5 /* limit */, 5 /* offset */);
>     FilterList filters = new FilterList(Operator.MUST_PASS_ALL);
>     filters.addFilter(filter1);
>     filters.addFilter(filter2);
>     Get get = new Get(USER);
>     get.addFamily(COLUMN_FAMILY);
>     get.setMaxVersions(1);
>     get.setFilter(filters);
> Somehow it didn't work as expected. It returned the entries as if the filter1 were not set.
> Turns out the ColumnPrefixFilter returns SEEK_NEXT_USING_HINT in some cases. The FilterList filter does not handle this return code properly (treat it as INCLUDE).

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira