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

Doğacan Güney commented on HBASE-1647:
--------------------------------------

Hey Ryan,

I did not mean that with my description but I was not clear at all. From IRC:

dogacan: St^Ack: "On StoreScanner running through in an column order rather 
than row-at-a-time, thats not how I understand it works but maybe thats how it 
appears in this context."
[10:32pm] dogacan: you are right here. I meant (again if I understood code 
correctly) scanners go to next row to figure out they went too far [i mean we 
peek to the next row in StoreScanner then get DONE then call filterRow]
[10:32pm] dogacan: and when they do, they used to call filterRow

([.....] part was not on IRC)

Anyway, did you try the ScanBug class I attached? When you set a ValueFilter, 
it filters out all other columns (because in current way there is almost one 
filterRow call for every filterKeyValue call).

> Filter#filterRow is called too often, filters rows it shouldn't have
> --------------------------------------------------------------------
>
>                 Key: HBASE-1647
>                 URL: https://issues.apache.org/jira/browse/HBASE-1647
>             Project: Hadoop HBase
>          Issue Type: Bug
>    Affects Versions: 0.20.0
>            Reporter: Doğacan Güney
>             Fix For: 0.20.0
>
>         Attachments: HBASE-1647-v2.patch, ScanBug.java, scanfilter.patch
>
>
> Filter#filterRow is called from ScanQueryMatcher#filterEntireRow which is 
> called from StoreScanner.next. However, if I understood the code correctly, 
> StoreScanner processes KeyValue-s in a column-oriented order (i.e. after 
> row1-col1 comes row2-col1, not row1-col2). Thus, when filterEntireRow is 
> called, in reality, the filter only processed (via filterKeyValue) only one 
> column of a row.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to