You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Anoop Sam John (JIRA)" <ji...@apache.org> on 2012/06/04 10:04:24 UTC

[jira] [Commented] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

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

Anoop Sam John commented on HBASE-6132:
---------------------------------------

I think that filterRow() in FilterList should not check with the filterAllRemaining() on individual filters. filterRow() is being called to check whether the current row under check needs to be included or not. filterAllRemaining() speaks abt the rows coming after the current row [In this context]
Note : filterAllRemaining() will be called during the read of individual KVs belonging to one row where it tells abt skipping the coming KVs.

Whatever KVs were included for one row should get returned by the scanner.  So usage of filterAllRemaining() as part of the filterRow() check should be avoided I feel.

Going through other method impls in FilterList. Getting doubts with some others also.
                
> ColumnCountGetFilter & PageFilter not working with FilterList
> -------------------------------------------------------------
>
>                 Key: HBASE-6132
>                 URL: https://issues.apache.org/jira/browse/HBASE-6132
>             Project: HBase
>          Issue Type: Bug
>          Components: filters
>    Affects Versions: 0.92.0, 0.92.1, 0.94.0
>         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
>            Reporter: Benjamin Kim
>
> Thanks to Anoop and Ramkrishna, here's what we found with FilterList
> If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.
> This problem seems to occur when specified column count is less then actual number of existing columns.
> Also same problem arises with PageFilter
> Following is the code of the problem:
> {code}
> Configuration conf = HBaseConfiguration.create();
> HTable table = new HTable(conf, "test");
> Get get = new Get(Bytes.toBytes("test00001"));
> FilterList filterList = new FilterList();
> filterList.addFilter(new ColumnCountGetFilter(100));           
> get.setFilter(filterList);
> Result r = table.get(get);
> System.out.println(r.size()); // prints zero
> {code}

--
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