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

Zheng Hu commented on HBASE-19729:
----------------------------------

After discussed with [~zghaobac] & [~Apache9],  still have some points to be 
optimized and fixed. 

{code}
-    return count > versionsAfterFilter ? MatchCode.SEEK_NEXT_COL : 
MatchCode.INCLUDE;
+
+    if (count > versionsAfterFilter) {
+      return MatchCode.SEEK_NEXT_COL;
+    } else {
{code}

when the number of cells exceed max version in scan, we should return 
SEEK_NEXT_COL match code, but if current code is INCLUDE_AND_SEEK_NEXT_ROW, we 
can optimize to choose the max step between SEEK_NEXT_COL and 
INCLUDE_AND_SEEK_NEXT_ROW, which is SEEK_NEXT_ROW.   So the code may be :

{code}
if (count > versionsAfterFilter) {
      if (matchCode == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
        matchCode = MatchCode.SEEK_NEXT_ROW;
      } else {
        matchCode = MatchCode.SEEK_NEXT_COL;
      }
   }
{code}

Other point need to consider  is :  when count > versionsAfterFilter,   the 
match code will be turned to SEEK_NEXT_COL,  and  here we forget to switch to 
the next column in ColumnTracker by columns#doneWithColumn(cell) , so may have 
the same problem as HBASE-19696 described.   

So finally, the  addendum fix will be  addendum.patch. The fix seems to be 
easy, but new UT for it is complex. Will upload the final addendum.patch with 
UT. 

> UserScanQueryMatcher#mergeFilterResponse should return 
> INCLUDE_AND_SEEK_NEXT_ROW when filterResponse is INCLUDE_AND_SEEK_NEXT_ROW
> ---------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: HBASE-19729
>                 URL: https://issues.apache.org/jira/browse/HBASE-19729
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Zheng Hu
>            Assignee: Zheng Hu
>              Labels: scanner
>             Fix For: 2.0.0-beta-2
>
>         Attachments: HBASE-19729.v1.patch, HBASE-19729.v2.patch, 
> HBASE-19729.v3.patch, HBASE-19729.v4.patch, HBASE-19729.v4.patch
>
>
> As we've discussed in HBASE-19696 
> https://issues.apache.org/jira/browse/HBASE-19696?focusedCommentId=16309644&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16309644
> when (filterResponse, matchCode) = (INCLUDE_AND_SEEK_NEXT_ROW, INCLUDE) or 
> (INCLUDE_AND_SEEK_NEXT_ROW, INCLUDE_AND_NEXT_COL) ,  we should return 
> INCLUDE_AND_SEEK_NEXT_ROW as the merged match code. 
> Will upload patches for all branches. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to