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

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

The problem is with the below code
In FilterList.java
{code}
public boolean filterRow() {
    for (Filter filter : filters) {
      if (operator == Operator.MUST_PASS_ALL) {
        if (filter.filterAllRemaining() || filter.filterRow()) {
          return true;
        }
      } else if (operator == Operator.MUST_PASS_ONE) {
        if (!filter.filterAllRemaining()
            && !filter.filterRow()) {
          return false;
        }
      }
    }
    return  operator == Operator.MUST_PASS_ONE;
  }
{code}
Here filterRow checks for filterAllRemaining() also.
In the issue case, the desirable number of KVs are included and remaining 
filtered out using the ColumnCountGetFilter. This filter implements the 
filterAllRemaining() also which returns true once the desirable number of KVs 
are included.  Later when the filterRow is called, as per the above code it 
will call the filterAllRemaining() on ColumnCountGetFilter and this row also 
will get avoided.

Same issue with PaginationFilter where the desired number of rows suppose 10.  
9 rows will get included and once 10th row is reached because of the  
filterAllRemaining() returns true from then onwards, the 10th row wont get 
included.

Why filterRow() impl of FilterList check with the filterAllRemaining() on the 
contained filters?
                
> 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:
> 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

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

        

Reply via email to