Return-Path: X-Original-To: apmail-hbase-issues-archive@www.apache.org Delivered-To: apmail-hbase-issues-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 5520F9591 for ; Mon, 4 Jun 2012 08:04:26 +0000 (UTC) Received: (qmail 34261 invoked by uid 500); 4 Jun 2012 08:04:26 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 34069 invoked by uid 500); 4 Jun 2012 08:04:25 -0000 Mailing-List: contact issues-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@hbase.apache.org Received: (qmail 34034 invoked by uid 99); 4 Jun 2012 08:04:24 -0000 Received: from issues-vm.apache.org (HELO issues-vm) (140.211.11.160) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 04 Jun 2012 08:04:24 +0000 Received: from isssues-vm.apache.org (localhost [127.0.0.1]) by issues-vm (Postfix) with ESMTP id A416D14285F for ; Mon, 4 Jun 2012 08:04:24 +0000 (UTC) Date: Mon, 4 Jun 2012 08:04:24 +0000 (UTC) From: "Anoop Sam John (JIRA)" To: issues@hbase.apache.org Message-ID: <216038345.32978.1338797064674.JavaMail.jiratomcat@issues-vm> In-Reply-To: <1616256293.14081.1338340283378.JavaMail.jiratomcat@issues-vm> Subject: [jira] [Commented] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HBASE-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13288396#comment-13288396 ] Anoop Sam John commented on HBASE-6132: --------------------------------------- I think that filterRow() in FilterList should not check with the filterAllRemaining() on individual filters. filterRow() is being called to check whether the current row under check needs to be included or not. filterAllRemaining() speaks abt the rows coming after the current row [In this context] Note : filterAllRemaining() will be called during the read of individual KVs belonging to one row where it tells abt skipping the coming KVs. Whatever KVs were included for one row should get returned by the scanner. So usage of filterAllRemaining() as part of the filterRow() check should be avoided I feel. Going through other method impls in FilterList. Getting doubts with some others also. > ColumnCountGetFilter & PageFilter not working with FilterList > ------------------------------------------------------------- > > Key: HBASE-6132 > URL: https://issues.apache.org/jira/browse/HBASE-6132 > Project: HBase > Issue Type: Bug > Components: filters > Affects Versions: 0.92.0, 0.92.1, 0.94.0 > Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3 > Reporter: Benjamin Kim > > Thanks to Anoop and Ramkrishna, here's what we found with FilterList > If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues. > This problem seems to occur when specified column count is less then actual number of existing columns. > Also same problem arises with PageFilter > Following is the code of the problem: > {code} > Configuration conf = HBaseConfiguration.create(); > HTable table = new HTable(conf, "test"); > Get get = new Get(Bytes.toBytes("test00001")); > FilterList filterList = new FilterList(); > filterList.addFilter(new ColumnCountGetFilter(100)); > get.setFilter(filterList); > Result r = table.get(get); > System.out.println(r.size()); // prints zero > {code} -- 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