You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Kannan Muthukkaruppan (Created) (JIRA)" <ji...@apache.org> on 2011/12/29 20:51:30 UTC

[jira] [Created] (HBASE-5104) FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT

FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT
---------------------------------------------------------------------------------------------------------

                 Key: HBASE-5104
                 URL: https://issues.apache.org/jira/browse/HBASE-5104
             Project: HBase
          Issue Type: Bug
            Reporter: Kannan Muthukkaruppan
            Assignee: Madhuwanti Vaidya


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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13277188#comment-13277188 ] 

Phabricator commented on HBASE-5104:
------------------------------------

madhuvaidya has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

INLINE COMMENTS
  src/main/java/org/apache/hadoop/hbase/client/Get.java:472 This was done to maintain inter-op if we are not using either the storeLimit or the storeOffset.

REVISION DETAIL
  https://reviews.facebook.net/D2799

To: madhuvaidya, lhofhansl, Kannan, tedyu, stack, todd, JIRA, jxcn01, mbautin
Cc: jxcn01, Liyin

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Zhihong Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13266252#comment-13266252 ] 

Zhihong Yu commented on HBASE-5104:
-----------------------------------

I got the same error reported by Hadoop QA when trying to apply the patch:
{code}
patch: **** malformed patch at line 285: Index: src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
{code}
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

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

Posted by "Zhihong Yu (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13179107#comment-13179107 ] 

Zhihong Yu commented on HBASE-5104:
-----------------------------------

As the above javadoc implies, the best place for efficient pagination support would be at the Scan/Get API level.
                
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13405471#comment-13405471 ] 

Hadoop QA commented on HBASE-5104:
----------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12534463/jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_15_15_30.patch
  against trunk revision .

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 11 new or modified tests.

    +1 hadoop2.0.  The patch compiles against the hadoop 2.0 profile.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    -1 javac.  The applied patch generated 5 javac compiler warnings (more than the trunk's current 4 warnings).

    -1 findbugs.  The patch appears to introduce 7 new Findbugs (version 1.3.9) warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

     -1 core tests.  The patch failed these unit tests:
                       org.apache.hadoop.hbase.security.token.TestZKSecretWatcher
                  org.apache.hadoop.hbase.master.TestHMasterRPCException

Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/2312//testReport/
Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/2312//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html
Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/2312//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html
Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/2312//console

This message is automatically generated.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-06-19_20_12_21.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_12_43_28.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_15_15_30.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13269115#comment-13269115 ] 

Phabricator commented on HBASE-5104:
------------------------------------

mbautin has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

  Michael, Jimmy: thanks for reviewing! See my responses inline.

INLINE COMMENTS
  src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java:386 Done.

  src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java:387 Done.
  src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java:931 Done.
  src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java:932 Done.
  src/main/java/org/apache/hadoop/hbase/client/Scan.java:638 Done.
  src/main/java/org/apache/hadoop/hbase/client/Get.java:471 Done.
  src/main/protobuf/Client.proto:49 Done.
  src/main/protobuf/Client.proto:50 Done.
  src/main/protobuf/Client.proto:199 Done.
  src/main/protobuf/Client.proto:200 Done.
  src/test/java/org/apache/hadoop/hbase/HTestConst.java:18 This is not a test, this is a collection of constants used in tests.

  I tried to save some typing, because the intended usage pattern is HTestConst.DEFAULT_{TABLE,CF,ROW,etc}... However, if you feel strongly about it, I can rename it to HTestConstants.

  src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java:60 Added region.close(). I am assuming that takes care of closing the HLog (correct me if I'm wrong).
  src/main/java/org/apache/hadoop/hbase/client/Get.java:212 Yes, this offset is only within a particular (row, CF) combination. It gets reset back to zero when we move to the next row/CF. Added this to javadoc.
  src/main/java/org/apache/hadoop/hbase/client/Result.java:177 Got rid of this method.

REVISION DETAIL
  https://reviews.facebook.net/D2799

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

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

Posted by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13179144#comment-13179144 ] 

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

I think scanner caching would take care of that. Even with the pagination filter, eventually you have to retrieve the rows at the client.
                
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable pagination mechanism

Posted by "Kannan Muthukkaruppan (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Kannan Muthukkaruppan updated HBASE-5104:
-----------------------------------------

    Description: 
Addendum:

Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is nota very clean way of supporting pagination.  Some of the problems with it are:

* Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
* When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
* ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.

Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:

1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].

Original Post:

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

  was:
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).

        Summary: Provide a reliable pagination mechanism  (was: FilterList doesn't work right with ColumnPaginationFilter)
    
> Provide a reliable pagination mechanism
> ---------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is nota very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT

Posted by "Zhihong Yu (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13177409#comment-13177409 ] 

Zhihong Yu commented on HBASE-5104:
-----------------------------------

Option #1 is easier to implement.

I prefer option #2 which is really interesting feature.

So we have two more JIRAs to work on :-)
                
> FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT
> ---------------------------------------------------------------------------------------------------------
>
>                 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable pagination mechanism

Posted by "stack (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13180188#comment-13180188 ] 

stack commented on HBASE-5104:
------------------------------

bq. Alternatively, there was some discussion about startRow/stopRow also allowing to specify a CF/columm. Would that work here? It would allow precise placement of a scan and might be a relatively simple change with more general applicability.

@Lars FYI, pagination can't cross rows because filters can't do state across rows because there is no way to carry the state across regions which you'd need if going between the end row of one region and the start of the next region (at least, not currently)
                
> Provide a reliable pagination mechanism
> ---------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is nota very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13258732#comment-13258732 ] 

Phabricator commented on HBASE-5104:
------------------------------------

mbautin has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

  Ping.

REVISION DETAIL
  https://reviews.facebook.net/D2799

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Phabricator updated HBASE-5104:
-------------------------------

    Attachment: D2799.5.patch

mbautin updated the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".
Reviewers: madhuvaidya, lhofhansl, Kannan, tedyu, stack, todd, JIRA, jxcn01

  Rebasing and addressing review comments.

REVISION DETAIL
  https://reviews.facebook.net/D2799

AFFECTED FILES
  src/main/java/org/apache/hadoop/hbase/client/Get.java
  src/main/java/org/apache/hadoop/hbase/client/Scan.java
  src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
  src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
  src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
  src/main/protobuf/Client.proto
  src/test/java/org/apache/hadoop/hbase/HTestConst.java
  src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
  src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

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

Posted by "Kannan Muthukkaruppan (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13180103#comment-13180103 ] 

Kannan Muthukkaruppan commented on HBASE-5104:
----------------------------------------------

@Lars- I am not sure I totally understood your suggestion with startRow. What you suggest seems to work when pagination is at the rowKey level. The case we need is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X and limit Y.


You wrote: <<Even with the pagination filter, eventually you have to retrieve the rows at the client.>> No, the filters are applied on the server side.
                
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13253981#comment-13253981 ] 

Phabricator commented on HBASE-5104:
------------------------------------

tedyu has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

  Nice work.

INLINE COMMENTS
  src/main/java/org/apache/hadoop/hbase/client/Get.java:470 The assignment isn't needed here, right ?
  src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java:395 rowOffset -> storeOffset
  src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide2.java:38 Add test category, please.

REVISION DETAIL
  https://reviews.facebook.net/D2799

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13273743#comment-13273743 ] 

Hadoop QA commented on HBASE-5104:
----------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12526586/D2799.6.patch
  against trunk revision .

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 7 new or modified tests.

    +1 hadoop23.  The patch compiles against the hadoop 0.23.x profile.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    -1 findbugs.  The patch appears to introduce 27 new Findbugs (version 1.3.9) warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

     -1 core tests.  The patch failed these unit tests:
                       org.apache.hadoop.hbase.regionserver.wal.TestLogRollingNoCluster
                  org.apache.hadoop.hbase.coprocessor.TestCoprocessorEndpoint

Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/1853//testReport/
Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/1853//artifact/trunk/patchprocess/newPatchFindbugsWarnings.html
Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/1853//console

This message is automatically generated.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13277182#comment-13277182 ] 

Phabricator commented on HBASE-5104:
------------------------------------

stack has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

  lgtm

INLINE COMMENTS
  src/main/java/org/apache/hadoop/hbase/client/Get.java:212 Will this be accurate if rows are inserted meantime (or deleted?).
  src/main/java/org/apache/hadoop/hbase/client/Get.java:201 This is great.  One day we should do size-based too.
  src/main/java/org/apache/hadoop/hbase/client/Get.java:472 Why not just write out our version as 3?  To save some bytes on wire?
  src/main/java/org/apache/hadoop/hbase/client/Scan.java:102 Doesn't Scan and Get share common ancestor?
  src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java:647 THanks for doing this.
  src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java:452 You need to add below to each classified test for classification to work

    @org.junit.Rule
    public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
      new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();

REVISION DETAIL
  https://reviews.facebook.net/D2799

To: madhuvaidya, lhofhansl, Kannan, tedyu, stack, todd, JIRA, jxcn01, mbautin
Cc: jxcn01, Liyin

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13261198#comment-13261198 ] 

Phabricator commented on HBASE-5104:
------------------------------------

mbautin has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

  All unit tests have passed. Could someone familiar with the protobuf stuff in trunk please take a look and accept?

REVISION DETAIL
  https://reviews.facebook.net/D2799

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

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

Posted by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13179042#comment-13179042 ] 

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

Yeah, filters are applied before we do the version counting. There's even a comment in ScanQueryMatcher as to why this needs to be done:
{code}
    /**
     * Filters should be checked before checking column trackers. If we do
     * otherwise, as was previously being done, ColumnTracker may increment its
     * counter for even that KV which may be discarded later on by Filter. This
     * would lead to incorrect results in certain cases.
     */
{code}

Hmm... Maybe there could be a special filter LastVersionFilter (or something), together with allowing ColumnPaginationFilter to wrap another filter (an idea that I like more, the more I think about it).
                
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Mikhail Bautin (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Mikhail Bautin updated HBASE-5104:
----------------------------------

    Status: Patch Available  (was: Open)
    
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Kannan Muthukkaruppan (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13181546#comment-13181546 ] 

Kannan Muthukkaruppan commented on HBASE-5104:
----------------------------------------------

@Lars/Stack: In 89-fb, we had done this for adding a reliable "limit" mechanism (this works per-CF/per-row). Madhu had implemented this. The rev is here: http://svn.apache.org/viewvc?view=revision&revision=1181562. [I don't think this is ported to trunk yet.] We were thinking of extending/doing something similar for "offset".

Lars: The startColumn type of approach doesn't work for cases for example when you are using a ColumnValueFilter instead of filter based on column names. [See my previous post.]

Already, when we specify attributes such as timerange() or add a CF or specific column names, it applies to each row. So one way to think of this is that limit/offset are also applicable within each row the Scan encounters. Most folks are going to use it for "Get" (single row scans), but there is no need to preclude the functionality from a multi-row Scan either.

This is the API that was added in 89-fb:
{code}
 /**
   * Set the maximum number of values to return per row per Column Family
   * @param limit the maximum number of values returned / row / CF
   */
public void setMaxResultsPerColumnFamily(int limit)
{code}

The thought was we could add something like:
{code}
 /**
   * Skip offset number of values to return per row per Column Family
   * @param offset number of values to be skipped per row / CF
   */
public void setOffsetPerColumnFamily(int offset)
{code}




                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Phabricator updated HBASE-5104:
-------------------------------

    Attachment: D2799.1.patch

mbautin requested code review of "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".
Reviewers: madhuvaidya, lhofhansl, Kannan, tedyu, stack, todd, JIRA

  Porting Madhu's fix for intra-row pagination to trunk. This is what we have in 89-fb just as a starting point (currently there are test failures).

TEST PLAN
  Run unit tests

REVISION DETAIL
  https://reviews.facebook.net/D2799

AFFECTED FILES
  src/main/java/org/apache/hadoop/hbase/client/Get.java
  src/main/java/org/apache/hadoop/hbase/client/Result.java
  src/main/java/org/apache/hadoop/hbase/client/Scan.java
  src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
  src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide2.java

MANAGE HERALD DIFFERENTIAL RULES
  https://reviews.facebook.net/herald/view/differential/

WHY DID I GET THIS EMAIL?
  https://reviews.facebook.net/herald/transcript/6369/

Tip: use the X-Herald-Rules header to filter Herald messages in your client.

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13269122#comment-13269122 ] 

Hadoop QA commented on HBASE-5104:
----------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12525758/D2799.5.patch
  against trunk revision .

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 7 new or modified tests.

    +1 hadoop23.  The patch compiles against the hadoop 0.23.x profile.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    +1 findbugs.  The patch does not introduce any new Findbugs (version 1.3.9) warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

     -1 core tests.  The patch failed these unit tests:
                       org.apache.hadoop.hbase.coprocessor.TestMasterObserver
                  org.apache.hadoop.hbase.coprocessor.TestCoprocessorEndpoint
                  org.apache.hadoop.hbase.regionserver.TestSplitTransactionOnCluster

Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/1782//testReport/
Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/1782//artifact/trunk/patchprocess/newPatchFindbugsWarnings.html
Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/1782//console

This message is automatically generated.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Mikhail Bautin (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Mikhail Bautin updated HBASE-5104:
----------------------------------

    Status: Patch Available  (was: Open)
    
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-06-19_20_12_21.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_12_43_28.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_15_15_30.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

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

Posted by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13179138#comment-13179138 ] 

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

It can already be done from the Scan API.
# set the startRow
# call next() N+1 times (N is the page size), remember the last key
# ... do what you need with the page (rows 1-N)...
# set the startRow for the next scan to the last key returned, goto 2.

No need to burden the Scan API with more options.

(It's possible that I am missing something here.)
                
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13261939#comment-13261939 ] 

Phabricator commented on HBASE-5104:
------------------------------------

jxcn01 has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

  Looks good, just some minor things.

INLINE COMMENTS
  src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java:386 Can we set it only if scan.getMaxResultsPerColumnFamily() >= 0?
  src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java:387 Can we set it only if the offset is > 0?
  src/main/java/org/apache/hadoop/hbase/client/Scan.java:638 Can we check: this.storeOffset > 0 || this.storeLimit > -1?
  I assume the offset should be position, and store limit is non-negative.

  The other choice is to add some checking in the corresponding set methods.
  src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java:931 ditto
  src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java:932 ditto
  src/main/protobuf/Client.proto:49 uint32 should be better, with no default. If it is not set, then it is -1.
  src/main/protobuf/Client.proto:50 uint32 is preferred, with no default.  If it is not set, then it is 0.
  src/main/protobuf/Client.proto:199 ditto
  src/main/protobuf/Client.proto:200 ditto
  src/main/java/org/apache/hadoop/hbase/client/Get.java:471 ditto

REVISION DETAIL
  https://reviews.facebook.net/D2799

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Mikhail Bautin (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Mikhail Bautin updated HBASE-5104:
----------------------------------

    Attachment: jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-06-19_20_12_21.patch
    
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-06-19_20_12_21.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

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

Posted by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org>.
    [ 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Hadoop QA (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13254501#comment-13254501 ] 

Hadoop QA commented on HBASE-5104:
----------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12522735/D2799.3.patch
  against trunk revision .

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 7 new or modified tests.

    -1 patch.  The patch command could not apply the patch.

Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/1535//console

This message is automatically generated.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT

Posted by "Kannan Muthukkaruppan (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13177405#comment-13177405 ] 

Kannan Muthukkaruppan commented on HBASE-5104:
----------------------------------------------

Additional note: The use of ColumnPaginationFilter AND ColumnPrefixFilter for the intended use case (i.e. to get next set of 5 thread ids for tag2) probably will not work even after this bug is fixed. I think once the bug is fixed for the above data set, the program should return 0 kvs.

Here's why:
  (select * from Tab where filter1 and filter2)
should be same as:
  (select * from Tab where filter1) INTERSECT (select * from Tab where filter2)

When separately applied, filter1 will return rows with tag1 prefix and filter2 will return rows with tag0 prefix (for Jiakai's example above) and the INTERSECTION will be the empty set.

The real confusion here seems to be because of the use of a filter for pagination. This seems odd. In normal SQL for example, pagination is not part of the WHERE clause but a separate special clause (as if it was being applied on the results of a sub-query).

 (select * from Tab 
  where  column  LIKE 'tag1%"
  LIMIT 5 OFFSET 5)

Possible ways of supporting this use case:

1) Don't use AND (via FilterList), but enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].

Thoughts?





                
> FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT
> ---------------------------------------------------------------------------------------------------------
>
>                 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

        

[jira] [Updated] (HBASE-5104) FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT

Posted by "Kannan Muthukkaruppan (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Kannan Muthukkaruppan updated HBASE-5104:
-----------------------------------------

    Attachment: testFilterList.rb

A test case illustrating the problem attached.
                
> FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT
> ---------------------------------------------------------------------------------------------------------
>
>                 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

        

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

Posted by "Zhihong Yu (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13179141#comment-13179141 ] 

Zhihong Yu commented on HBASE-5104:
-----------------------------------

The above description assumes N is small which is probably true.
Normally it is suboptimal for scan to return just one row per call.

Let's see what Jiakai has to say.
                
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Phabricator updated HBASE-5104:
-------------------------------

    Attachment: D2799.6.patch

mbautin updated the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".
Reviewers: madhuvaidya, lhofhansl, Kannan, tedyu, stack, todd, JIRA, jxcn01

  Rebasing on trunk changes

REVISION DETAIL
  https://reviews.facebook.net/D2799

AFFECTED FILES
  src/main/java/org/apache/hadoop/hbase/client/Get.java
  src/main/java/org/apache/hadoop/hbase/client/Scan.java
  src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
  src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
  src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
  src/main/protobuf/Client.proto
  src/test/java/org/apache/hadoop/hbase/HTestConst.java
  src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
  src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java

To: madhuvaidya, lhofhansl, Kannan, tedyu, stack, todd, JIRA, jxcn01, mbautin
Cc: jxcn01

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13273747#comment-13273747 ] 

Phabricator commented on HBASE-5104:
------------------------------------

mbautin has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

  Could someone please accept this? This diff is almost a month old.

REVISION DETAIL
  https://reviews.facebook.net/D2799

To: madhuvaidya, lhofhansl, Kannan, tedyu, stack, todd, JIRA, jxcn01, mbautin
Cc: jxcn01

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Hadoop QA (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13253979#comment-13253979 ] 

Hadoop QA commented on HBASE-5104:
----------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12522665/D2799.2.patch
  against trunk revision .

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 2 new or modified tests.

    -1 patch.  The patch command could not apply the patch.

Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/1523//console

This message is automatically generated.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Mikhail Bautin (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Mikhail Bautin updated HBASE-5104:
----------------------------------

    Status: Open  (was: Patch Available)
    
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-06-19_20_12_21.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_12_43_28.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_15_15_30.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT

Posted by "Kannan Muthukkaruppan (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13177430#comment-13177430 ] 

Kannan Muthukkaruppan commented on HBASE-5104:
----------------------------------------------

Lars: I think viewing it as a general filter is problematic. Note: a FilterList wraps other filters, but it is perfectly legal for it to be wrapped by outer level filters... (say if you want to build a complex boolean expression). Whereas the pagination functionality doesn't lend itself to this flexibility. It is more like a special post-processing -only filter that you apply at the end stage. 

                
> FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT
> ---------------------------------------------------------------------------------------------------------
>
>                 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Zhihong Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13258742#comment-13258742 ] 

Zhihong Yu commented on HBASE-5104:
-----------------------------------

The previous Hadoop QA run stumbled over TestLoadIncrementalHFilesSplitRecovery
Please resubmit patch for QA.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Phabricator updated HBASE-5104:
-------------------------------

    Attachment: D2799.2.patch

mbautin updated the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".
Reviewers: madhuvaidya, lhofhansl, Kannan, tedyu, stack, todd, JIRA

  Actually adding the new fields into the protobuf. Still have one test method to fix in TestFromClientSide2 that fails, and the full test suite is still running.

REVISION DETAIL
  https://reviews.facebook.net/D2799

AFFECTED FILES
  src/main/java/org/apache/hadoop/hbase/client/Get.java
  src/main/java/org/apache/hadoop/hbase/client/Result.java
  src/main/java/org/apache/hadoop/hbase/client/Scan.java
  src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
  src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
  src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
  src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
  src/main/protobuf/Client.proto
  src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide2.java

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable pagination mechanism

Posted by "stack (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13180211#comment-13180211 ] 

stack commented on HBASE-5104:
------------------------------

@Lars Is this what Dave Revell was going to add at one point till we discouraged him by telling him use a filter?
                
> Provide a reliable pagination mechanism
> ---------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is nota very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

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

Posted by "Kannan Muthukkaruppan (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13179014#comment-13179014 ] 

Kannan Muthukkaruppan commented on HBASE-5104:
----------------------------------------------

Jiakai also reported that:

<<< There seems to be another problem with ColumnPaginationFilter: it seems to count multiple versions of a cell as multiple entries. i.e. when I first applied the ColumnPaginationFilter(10, 0), it returned the first 10 entries in that CF. Then I wrote the same set of test data to the CF and ran the query again, it only returned the first 5 entries. The table was created with maxVersion = 1. I also set Get's maxVersion to 1. However, I repeated the process several times (over write + query), it kept returning the top 5 entries (not like 5/2 entries, 5/4 entries if it really kept multiple versions). I could repo this with a new table. Is it expected behavior?>>>

Looks like there are additional problems with doing pagination support as a filter.


                
> 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

        

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

Posted by "Kannan Muthukkaruppan (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13177451#comment-13177451 ] 

Kannan Muthukkaruppan commented on HBASE-5104:
----------------------------------------------

Lars: Yes.

Jiakai wrote in with: <<< the filters in FilterList are applied in order. The ColumnPaginationFilter's filterKeyValue() is called only when ColumnPrefixFilter's filterKeyValue() returns true. i.e. the current implementation should be equivalent to:
select * from (select * from Tab where filter1) where filter2

So it should return the desired result after the bug is fixed.

If you meant to suggest that filters in FilterList should be interchangeable, then it becomes a design question. I'm fine with the alternative approaches you suggested, too.>>>>

Response:  Existing code structure wise, Jiakai is correct. The filters are evaluated in order... so once SEEK_NEXT_USING_HINT is correctly handled, you'll get the behavior you want. But I am concerned overall with a ColumnPaginationFilter being a stateful filter whose state gets updated depending on what other filters where ahead of it. But perhaps, for backward compatibility, we cannot change its existing behavior.

So we'll probably need to do both... fix the SEEK_NEXT_USING_HINT to work right with FilterList (at which point your case will start working fine), and also support limit/offset at the Scan/Get or ColumnPrefixFilter level as a cleaner alternative to do pagination.

One disadvantage of sticking with the FilterList approach would be that it might be trickier to get the "seek_next_using_hint" optimization. The ColumnPrefixFilter can only seek next using hint in limited circumstances. For example, if you have an OR filter of two prefix filters:

((ColumnPrefix("B") or ColumnPrefix("A")) AND (PaginationFilter(5, 5))

we cannot have the first filter suggest a SEEK_NEXT_USING_HINT to go to prefix B, as that'll miss out columns starting at "A".

We'll need to restrict the SEEK_NEXT_USING_HINT to be used in much more limited circumstances... and if there are other filters in the mix, we probably need to scan one cell at a time. This might be another reason to deal with LIMIT/OFFSET as either an option to the ColumnPrefixFilter itself or at the Scan/Get API level.

                
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13254498#comment-13254498 ] 

Phabricator commented on HBASE-5104:
------------------------------------

mbautin has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

INLINE COMMENTS
  src/main/java/org/apache/hadoop/hbase/client/Get.java:470 Removed.
  src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java:395 Done.
  src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide2.java:38 Done.

REVISION DETAIL
  https://reviews.facebook.net/D2799

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13270027#comment-13270027 ] 

Phabricator commented on HBASE-5104:
------------------------------------

mbautin has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

  All unit tests pass, except TestCoprocessorEndpoint, which fails without this patch too. Could someone take another look and accept? Thanks!

REVISION DETAIL
  https://reviews.facebook.net/D2799

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13277168#comment-13277168 ] 

Phabricator commented on HBASE-5104:
------------------------------------

Liyin has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

  LGTM ! Thanks Mikhail !

REVISION DETAIL
  https://reviews.facebook.net/D2799

To: madhuvaidya, lhofhansl, Kannan, tedyu, stack, todd, JIRA, jxcn01, mbautin
Cc: jxcn01, Liyin

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Mikhail Bautin (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Mikhail Bautin updated HBASE-5104:
----------------------------------

    Resolution: Fixed
        Status: Resolved  (was: Patch Available)
    
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-06-19_20_12_21.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_12_43_28.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_15_15_30.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13261168#comment-13261168 ] 

Hadoop QA commented on HBASE-5104:
----------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12524087/D2799.4.patch
  against trunk revision .

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 7 new or modified tests.

    -1 patch.  The patch command could not apply the patch.

Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/1635//console

This message is automatically generated.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

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

Posted by "Kannan Muthukkaruppan (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Kannan Muthukkaruppan updated HBASE-5104:
-----------------------------------------

    Summary: FilterList doesn't work right with ColumnPaginationFilter  (was: FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT)
    
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Phabricator updated HBASE-5104:
-------------------------------

    Attachment: D2799.4.patch

mbautin updated the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".
Reviewers: madhuvaidya, lhofhansl, Kannan, tedyu, stack, todd, JIRA

  Rebasing on changes in trunk

REVISION DETAIL
  https://reviews.facebook.net/D2799

AFFECTED FILES
  src/main/java/org/apache/hadoop/hbase/client/Get.java
  src/main/java/org/apache/hadoop/hbase/client/Result.java
  src/main/java/org/apache/hadoop/hbase/client/Scan.java
  src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
  src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
  src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
  src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
  src/main/protobuf/Client.proto
  src/test/java/org/apache/hadoop/hbase/HTestConst.java
  src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
  src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

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

Posted by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13177442#comment-13177442 ] 

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

And of course this bug still needs to be fixed.
                
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Mikhail Bautin (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Mikhail Bautin updated HBASE-5104:
----------------------------------

    Attachment: jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch

Manually attaching the most recent patch.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Zhihong Yu (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13181552#comment-13181552 ] 

Zhihong Yu commented on HBASE-5104:
-----------------------------------

@Kannan:
The above methods look good.
Minor suggestion: I think the method names should reflect the nature of intra-row pagination. Currently people need to read the javadoc to get that.
Basically we should distinguish this feature from inter-row pagination support.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Phabricator updated HBASE-5104:
-------------------------------

    Attachment: D2799.3.patch

mbautin updated the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".
Reviewers: madhuvaidya, lhofhansl, Kannan, tedyu, stack, todd, JIRA

  Fixing a bug with StoreScanner not resetting the current offset within the row/column family (countPerRow) to zero when transitioning to the next row. This is a porting error that is not present in Madhu's original fix for 89-fb. Also adding an independent test based on HRegion API that helped catch this bug (TestIntraRowPagination). I will post additional test results later.

REVISION DETAIL
  https://reviews.facebook.net/D2799

AFFECTED FILES
  src/main/java/org/apache/hadoop/hbase/client/Get.java
  src/main/java/org/apache/hadoop/hbase/client/Result.java
  src/main/java/org/apache/hadoop/hbase/client/Scan.java
  src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
  src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
  src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
  src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
  src/main/protobuf/Client.proto
  src/test/java/org/apache/hadoop/hbase/HTestConst.java
  src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
  src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13266246#comment-13266246 ] 

Phabricator commented on HBASE-5104:
------------------------------------

stack has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

  Good stuff.

INLINE COMMENTS
  src/main/java/org/apache/hadoop/hbase/client/Get.java:212 How does this work?  We have to run through the row by column family?  We set this offset back to zero when we move to a new column family on a row?
  src/main/java/org/apache/hadoop/hbase/client/Result.java:177 Why we need this?  Isn't the Result sorted already?  If not, its a bug.
  src/test/java/org/apache/hadoop/hbase/HTestConst.java:18 Why is this test not called HTestConstants.java
  src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java:60 Should close the region when done and close out its hlog.

REVISION DETAIL
  https://reviews.facebook.net/D2799

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable pagination mechanism

Posted by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13180219#comment-13180219 ] 

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

You mean HBASE-4256? Yep... :)
Although we left it there with saying that intra-row scanning is useful, but that his use case can be solved with a ColumnRangeFilter.

                
> Provide a reliable pagination mechanism
> ---------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is nota very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Mikhail Bautin (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Mikhail Bautin updated HBASE-5104:
----------------------------------

    Attachment: jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_15_15_30.patch
    
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-06-19_20_12_21.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_12_43_28.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_15_15_30.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable pagination mechanism

Posted by "Kannan Muthukkaruppan (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13181013#comment-13181013 ] 

Kannan Muthukkaruppan commented on HBASE-5104:
----------------------------------------------

Yes, the case we are looking at was indeed really intra-row pagination. I think I should update the JIRA title to reflect that.
                
> Provide a reliable pagination mechanism
> ---------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is nota very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Hadoop QA (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13253971#comment-13253971 ] 

Hadoop QA commented on HBASE-5104:
----------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12522655/D2799.1.patch
  against trunk revision .

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 2 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    -1 findbugs.  The patch appears to introduce 4 new Findbugs (version 1.3.9) warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

     -1 core tests.  The patch failed these unit tests:
     

Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/1522//testReport/
Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/1522//artifact/trunk/patchprocess/newPatchFindbugsWarnings.html
Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/1522//console

This message is automatically generated.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Jimmy Xiang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13261941#comment-13261941 ] 

Jimmy Xiang commented on HBASE-5104:
------------------------------------

I commented on phabricator.  Looks good to me, just some minor things.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT

Posted by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13177425#comment-13177425 ] 

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

It seems to me that ColumnPaginationFilter should wrap another filter (similar to how FilterList wraps other filters), and then apply the pagination on the result (the wrapper filter of course could be another filter list).

                
> FilterList doesn't work right with filters (such as ColumPrefixFilter) which use the SEEK_NEXT_USING_HINT
> ---------------------------------------------------------------------------------------------------------
>
>                 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Mikhail Bautin (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13406128#comment-13406128 ] 

Mikhail Bautin commented on HBASE-5104:
---------------------------------------

Committed.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-06-19_20_12_21.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_12_43_28.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_15_15_30.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Kannan Muthukkaruppan (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13181020#comment-13181020 ] 

Kannan Muthukkaruppan commented on HBASE-5104:
----------------------------------------------

<<<I just had a discussion today about how it would be nice if one could start a scanner at a certain column prefix within a certain row and also set a stop column prefix with in a row. (i.e. not using a filter).>>>

Why would you not want to use a filter for this case? ColumnRangeFilter() handles this case nicely correct?


<<< Alternatively, there was some discussion about startRow/stopRow also allowing to specify a CF/columm. Would that work here? It would allow precise placement of a scan and might be a relatively simple change with more general applicability.>>>

This may not work for many cases. How do I, for instance say, get me the next 5 KVs in a particular row whose value is X (note: here the filter is on column value rather than column name; assume you are using the SingleColumnValueFilter()).

I think limit/offset is a simple/well understood concept that we should support in a clean way. Scan/Get API seems like a good place to do it. What is the concern with adding the capability there?

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Zhihong Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13277055#comment-13277055 ] 

Zhihong Yu commented on HBASE-5104:
-----------------------------------

I stumbled on the following test failure twice (with D2799.6.patch on MacBook):
{code}
testExecDeserialization(org.apache.hadoop.hbase.coprocessor.TestCoprocessorEndpoint)  Time elapsed: 0.028 sec  <<< ERROR!
java.io.EOFException
  at java.io.DataInputStream.readFully(DataInputStream.java:180)
  at java.io.DataInputStream.readUTF(DataInputStream.java:592)
  at java.io.DataInputStream.readUTF(DataInputStream.java:547)
  at org.apache.hadoop.hbase.client.coprocessor.Exec.readFields(Exec.java:120)
  at org.apache.hadoop.hbase.coprocessor.TestCoprocessorEndpoint.testExecDeserialization(TestCoprocessorEndpoint.java:201)
{code}
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13397242#comment-13397242 ] 

Hadoop QA commented on HBASE-5104:
----------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12532637/jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-06-19_20_12_21.patch
  against trunk revision .

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 11 new or modified tests.

    +1 hadoop2.0.  The patch compiles against the hadoop 2.0 profile.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    -1 findbugs.  The patch appears to introduce 7 new Findbugs (version 1.3.9) warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

     -1 core tests.  The patch failed these unit tests:
                       org.apache.hadoop.hbase.io.hfile.TestForceCacheImportantBlocks
                  org.apache.hadoop.hbase.coprocessor.TestCoprocessorEndpoint
                  org.apache.hadoop.hbase.security.access.TestZKPermissionsWatcher

Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/2194//testReport/
Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/2194//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html
Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/2194//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html
Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/2194//console

This message is automatically generated.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-06-19_20_12_21.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

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

Posted by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13180117#comment-13180117 ] 

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

@Kannan:
Re startRow: I see, thanks for explaining.
Re filters on server: Of course... What I meant was that eventually the cells you care for (after filtering) are retrieved by the client.

I would still prefer a ColumnPaginationFilter that can wrap another filter over a Limit/Offset API change to the scan API.

Alternatively, there was some discussion about startRow/stopRow also allowing to specify a CF/columm. Would that work here? It would allow precise placement of a scan and might be a relatively simple change with more general applicability.

                
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13190892#comment-13190892 ] 

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

After spending some time thinking about HBASE-5229, I think this use case can be addressed by
(1) Allowing ColumnPaginationFilter to wrap another filter (similar to WhileMatchFilter) and
(2) Allowing filter to be optionally evaluated after we handled versions.

For #2 either each filter could carry a flag, or we have another filter wrapper to indicate after versioning evaluation.
I realize I sound like a broken record, but that would handle a more general set of use cases (including this one, but correct me if I am wrong, Kannan), and also avoid adding special case API to the scanning API.

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Kannan Muthukkaruppan (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Kannan Muthukkaruppan updated HBASE-5104:
-----------------------------------------

    Description: 
Addendum:

Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:

* Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
* When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
* ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.

Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:

1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].

Original Post:

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

  was:
Addendum:

Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is nota very clean way of supporting pagination.  Some of the problems with it are:

* Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
* When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
* ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.

Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:

1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].

Original Post:

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

        Summary: Provide a reliable intra-row pagination mechanism  (was: Provide a reliable pagination mechanism)
    
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13181053#comment-13181053 ] 

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

It seems you have a very specific usecase. A limit/offset API that is column based on an API that is inherently row based (scanner.next) will be hard to understand for users.

The problem here seems to be that scanner.startRow and scanner.next do not provide enough granularity.

I'm not opposed to limit/offset (but I will be interested to see how you will document that API, to make is understandable to users :) ).

What about a nextColumn method on scanner along with a startColumn method?

Anyway... I just want to make sure we do not add API for specific cases, and I'll shut up about it now.

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Hadoop QA (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13255008#comment-13255008 ] 

Hadoop QA commented on HBASE-5104:
----------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12522843/jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch
  against trunk revision .

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 11 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    -1 findbugs.  The patch appears to introduce 4 new Findbugs (version 1.3.9) warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

     -1 core tests.  The patch failed these unit tests:
     

Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/1541//testReport/
Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/1541//artifact/trunk/patchprocess/newPatchFindbugsWarnings.html
Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/1541//console

This message is automatically generated.
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Zhihong Yu (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13254820#comment-13254820 ] 

Zhihong Yu commented on HBASE-5104:
-----------------------------------

Patch didn't apply cleanly:
{code}
/usr/bin/patch: **** malformed patch at line 285: Index: src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
{code}
                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Phabricator (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13259787#comment-13259787 ] 

Phabricator commented on HBASE-5104:
------------------------------------

madhuvaidya has commented on the revision "[jira] [HBASE-5104] Provide a reliable intra-row pagination mechanism".

  LGTM (at least all the non-protocol buffer related stuff).

REVISION DETAIL
  https://reviews.facebook.net/D2799

                
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Updated] (HBASE-5104) Provide a reliable intra-row pagination mechanism

Posted by "Mikhail Bautin (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Mikhail Bautin updated HBASE-5104:
----------------------------------

    Attachment: jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_12_43_28.patch
    
> Provide a reliable intra-row pagination mechanism
> -------------------------------------------------
>
>                 Key: HBASE-5104
>                 URL: https://issues.apache.org/jira/browse/HBASE-5104
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Kannan Muthukkaruppan
>            Assignee: Madhuwanti Vaidya
>         Attachments: D2799.1.patch, D2799.2.patch, D2799.3.patch, D2799.4.patch, D2799.5.patch, D2799.6.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-04-16_12_39_42.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-06-19_20_12_21.patch, jira-HBASE-5104-Provide-a-reliable-intra-row-paginat-2012-07-02_12_43_28.patch, testFilterList.rb
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is not a very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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

        

[jira] [Commented] (HBASE-5104) Provide a reliable pagination mechanism

Posted by "Lars Hofhansl (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13180201#comment-13180201 ] 

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

Thanks Stack. I see, so ColumnPaginationFilter cannot work if we want to cross rows. I think what Kannan has in mind is pagination within a given row (from the description).

Having more precise control of scanner start and stop cell might be nice anyway. I just had a discussion today about how it would be nice if one could start a scanner at a certain column prefix within a certain row and also set a stop column prefix with in a row. (i.e. not using a filter). It seems this would be generally applicable and also solve Kannan's use case. Correct Kannan?

Something like Scan.setStartRow(byte[] rowkey, byte[] column), which would request to seek the scanner to that exact column prefix (while honoring all version settings for the scanner)... Same for setStopRow(byte[] rowkey, byte[] column).

                
> Provide a reliable pagination mechanism
> ---------------------------------------
>
>                 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
>
>
> Addendum:
> Doing pagination (retrieving at most "limit" number of KVs at a particular "offset") is currently supported via the ColumnPaginationFilter. However, it is nota very clean way of supporting pagination.  Some of the problems with it are:
> * Normally, one would expect a query with (Filter(A) AND Filter(B)) to have same results as (query with Filter(A)) INTERSECT (query with Filter(B)). This is not the case for ColumnPaginationFilter as its internal state gets updated depending on whether or not Filter(A) returns TRUE/FALSE for a particular cell.
> * When this Filter is used in combination with other filters (e.g., doing AND with another filter using FilterList), the behavior of the query depends on the order of filters in the FilterList. This is not ideal.
> * ColumnPaginationFilter is a stateful filter which ends up counting multiple versions of the cell as separate values even if another filter upstream or the ScanQueryMatcher is going to reject the value for other reasons.
> Seems like we need a reliable way to do pagination. The particular use case that prompted this JIRA is pagination within the same rowKey. For example, for a given row key R, get columns with prefix P, starting at offset X (among columns which have prefix P) and limit Y. Some possible fixes might be:
> 1) enhance ColumnPrefixFilter to support another constructor which supports limit/offset.
> 2) Support pagination (limit/offset) at the Scan/Get API level (rather than as a filter) [Like SQL].
> Original Post:
> 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