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

2012-01-03 Thread Kannan Muthukkaruppan (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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

2012-01-03 Thread Lars Hofhansl (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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] [Commented] (HBASE-5104) FilterList doesn't work right with ColumnPaginationFilter

2012-01-03 Thread Zhihong Yu (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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) FilterList doesn't work right with ColumnPaginationFilter

2012-01-03 Thread Lars Hofhansl (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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) FilterList doesn't work right with ColumnPaginationFilter

2012-01-03 Thread Zhihong Yu (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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] [Commented] (HBASE-5104) FilterList doesn't work right with ColumnPaginationFilter

2012-01-03 Thread Lars Hofhansl (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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] [Commented] (HBASE-5104) FilterList doesn't work right with ColumnPaginationFilter

2011-12-29 Thread Lars Hofhansl (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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) FilterList doesn't work right with ColumnPaginationFilter

2011-12-29 Thread Lars Hofhansl (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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] [Commented] (HBASE-5104) FilterList doesn't work right with ColumnPaginationFilter

2011-12-29 Thread Kannan Muthukkaruppan (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-5104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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