hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Anoop Sam John (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-10850) Unexpected behavior when using filter SingleColumnValueFilter
Date Wed, 02 Apr 2014 17:19:16 GMT

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

Anoop Sam John commented on HBASE-10850:
----------------------------------------

No.. See the change in FilterWrapper
{code}
   public void filterRowCells(List<Cell> kvs) throws IOException {
-    //To fix HBASE-6429,
-    //Filter with filterRow() returning true is incompatible with scan with limit
-    //1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented.
-    //2. filterRow() is merged with filterRow(kvs),
-    //so that to make all those row related filtering stuff in the same function.
     this.filter.filterRowCells(kvs);
-    if (!kvs.isEmpty() && this.filter.filterRow()) {
-      kvs.clear();
-    }
   }
{code}

> Unexpected behavior when using filter SingleColumnValueFilter
> -------------------------------------------------------------
>
>                 Key: HBASE-10850
>                 URL: https://issues.apache.org/jira/browse/HBASE-10850
>             Project: HBase
>          Issue Type: Bug
>          Components: Filters
>    Affects Versions: 0.96.1.1
>            Reporter: Fabien Le Gallo
>            Assignee: Ted Yu
>            Priority: Critical
>         Attachments: 10850-hasFilterRow-v1.txt, 10850-hasFilterRow-v2.txt, 10850-hasFilterRow-v3.txt,
10850-v4.txt, HBASE-10850-96.patch, HBASE-10850.patch, HBASE-10850_V2.patch, HBaseSingleColumnValueFilterTest.java,
TestWithMiniCluster.java
>
>
> When using the filter SingleColumnValueFilter, and depending of the columns specified
in the scan (filtering column always specified), the results can be different.
> Here is an example.
> Suppose the following table:
> ||key||a:foo||a:bar||b:foo||b:bar||
> |1|false|_flag_|_flag_|_flag_|
> |2|true|_flag_|_flag_|_flag_|
> |3| |_flag_|_flag_|_flag_|
> With this filter:
> {code}
> SingleColumnValueFilter filter = new SingleColumnValueFilter(Bytes.toBytes("a"), Bytes.toBytes("foo"),
CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("false")));
> filter.setFilterIfMissing(true);
> {code}
> Depending of how I specify the list of columns to add in the scan, the result is different.
Yet, all examples below should always return only the first row (key '1'):
> OK:
> {code}
> scan.addFamily(Bytes.toBytes("a"));
> {code}
> KO (2 results returned, row '3' without 'a:foo' qualifier is returned):
> {code}
> scan.addFamily(Bytes.toBytes("a"));
> scan.addFamily(Bytes.toBytes("b"));
> {code}
> KO (2 results returned, row '3' without 'a:foo' qualifier is returned):
> {code}
> scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("foo"));
> scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("bar"));
> scan.addColumn(Bytes.toBytes("b"), Bytes.toBytes("foo"));
> {code}
> OK:
> {code}
> scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("foo"));
> scan.addColumn(Bytes.toBytes("b"), Bytes.toBytes("bar"));
> {code}
> OK:
> {code}
> scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("foo"));
> scan.addColumn(Bytes.toBytes("a"), Bytes.toBytes("bar"));
> {code}
> This is a regression as it was working properly on HBase 0.92.
> You will find in attachement the unit tests reproducing the issue.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Mime
View raw message