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

takeshi.miao commented on HBASE-10290:
--------------------------------------

I think that I can help on this issue, but still need to confirm the behaviours 
the HBase shall do... 
For example, 
1. I have 5 rows of data
|row|cf1:c1|cf1:c2|cf2:c3|
|001|AA|BB| |
|002|CC| |DD|
|003|EE| |FF|
|004|CC| |HH|
|005|II|JJ| |

2. Then I use following code to get data from HBase
{code}
//...
fl = new FilterList(Operator.MUST_PASS_ALL);

filter =
    new SingleColumnValueFilter(Bytes.toBytes("cf1"), Bytes.toBytes("c1"), 
CompareOp.EQUAL,
        Bytes.toBytes("CC"));
fl.addFilter(filter);

filter = new QualifierFilter(CompareOp.EQUAL, new 
BinaryComparator(Bytes.toBytes("c3")));
fl.addFilter(filter);

scan.setFilter(fl);
rs = table.getScanner(scan);

for (Result r : rs) {
  //...
}
{code}

3. So I shall get the two rows of data returned from HBase, right ?
|row|cf1:c1|cf1:c2|cf2:c3|
|002|CC| |DD|
|004|CC| |HH|

4. Otherwise, if I use _Operator.MUST_PASS_ONE_, I shall get three rows of data 
returned from HBase, right ?
|row|cf1:c1|cf1:c2|cf2:c3|
|002|CC| |DD|
|003|EE| |FF|
|004|CC| |HH|


> The results are not expected by using QualifierFilter combined with 
> SingleColumnValueFilter
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-10290
>                 URL: https://issues.apache.org/jira/browse/HBASE-10290
>             Project: HBase
>          Issue Type: Bug
>          Components: Filters
>    Affects Versions: 0.99.0
>            Reporter: takeshi.miao
>         Attachments: TestFilters.java
>
>
> I suffering an issue is, I only get one _KeyValue_ object (five expected) 
> from one row if I use _FilterList_ with _Operator.MUST_PASS_ALL_ to add both 
> _QualifierFiter_ and _Operator.MUST_PASS_ALL_, the details as follows
> h4.Test for Operator.MUST_PASS_ALL
> 1. I generate 10 rows of test data, two column familys and five column 
> qualifiers for each row.
> {code:title=Test Data}
> r=keyvalues={row001/cf1:dummy/1389104461530/Put/vlen=5/mvcc=0, 
> row001/cf1:ip/1389104461530/Put/vlen=10/mvcc=0, 
> row001/cf1:port-10/1389104461530/Put/vlen=5/mvcc=0, 
> row001/cf2:dummy1/1389104461530/Put/vlen=6/mvcc=0, 
> row001/cf2:dummy2/1389104461530/Put/vlen=6/mvcc=0}
> r=keyvalues={row002/cf1:dummy/1389104461530/Put/vlen=5/mvcc=0, 
> row002/cf1:ip/1389104461530/Put/vlen=10/mvcc=0, 
> row002/cf1:port-20/1389104461530/Put/vlen=5/mvcc=0, 
> row002/cf2:dummy1/1389104461530/Put/vlen=6/mvcc=0, 
> row002/cf2:dummy2/1389104461530/Put/vlen=6/mvcc=0}
> r=keyvalues={row003/cf1:dummy/1389104461530/Put/vlen=5/mvcc=0, 
> row003/cf1:ip/1389104461530/Put/vlen=10/mvcc=0, 
> row003/cf1:port-30/1389104461530/Put/vlen=5/mvcc=0, 
> row003/cf2:dummy1/1389104461530/Put/vlen=6/mvcc=0, 
> row003/cf2:dummy2/1389104461530/Put/vlen=6/mvcc=0}
> r=keyvalues={row004/cf1:dummy/1389104461530/Put/vlen=5/mvcc=0, 
> row004/cf1:ip/1389104461530/Put/vlen=10/mvcc=0, 
> row004/cf1:port-40/1389104461530/Put/vlen=5/mvcc=0, 
> row004/cf2:dummy1/1389104461530/Put/vlen=6/mvcc=0, 
> row004/cf2:dummy2/1389104461530/Put/vlen=6/mvcc=0}
> r=keyvalues={row005/cf1:dummy/1389104461530/Put/vlen=5/mvcc=0, 
> row005/cf1:ip/1389104461530/Put/vlen=10/mvcc=0, 
> row005/cf1:port-50/1389104461530/Put/vlen=5/mvcc=0, 
> row005/cf2:dummy1/1389104461530/Put/vlen=6/mvcc=0, 
> row005/cf2:dummy2/1389104461530/Put/vlen=6/mvcc=0}
> r=keyvalues={row006/cf1:dummy/1389104461530/Put/vlen=5/mvcc=0, 
> row006/cf1:ip/1389104461530/Put/vlen=10/mvcc=0, 
> row006/cf1:port-60/1389104461530/Put/vlen=5/mvcc=0, 
> row006/cf2:dummy1/1389104461530/Put/vlen=6/mvcc=0, 
> row006/cf2:dummy2/1389104461530/Put/vlen=6/mvcc=0}
> r=keyvalues={row007/cf1:dummy/1389104461530/Put/vlen=5/mvcc=0, 
> row007/cf1:ip/1389104461530/Put/vlen=10/mvcc=0, 
> row007/cf1:port-70/1389104461530/Put/vlen=5/mvcc=0, 
> row007/cf2:dummy1/1389104461530/Put/vlen=6/mvcc=0, 
> row007/cf2:dummy2/1389104461530/Put/vlen=6/mvcc=0}
> r=keyvalues={row008/cf1:dummy/1389104461530/Put/vlen=5/mvcc=0, 
> row008/cf1:ip/1389104461530/Put/vlen=10/mvcc=0, 
> row008/cf1:port-80/1389104461530/Put/vlen=5/mvcc=0, 
> row008/cf2:dummy1/1389104461530/Put/vlen=6/mvcc=0, 
> row008/cf2:dummy2/1389104461530/Put/vlen=6/mvcc=0}
> r=keyvalues={row009/cf1:dummy/1389104461530/Put/vlen=5/mvcc=0, 
> row009/cf1:ip/1389104461530/Put/vlen=10/mvcc=0, 
> row009/cf1:port-80/1389104461530/Put/vlen=5/mvcc=0, 
> row009/cf2:dummy1/1389104461530/Put/vlen=6/mvcc=0, 
> row009/cf2:dummy2/1389104461530/Put/vlen=6/mvcc=0}
> r=keyvalues={row010/cf1:dummy/1389104461530/Put/vlen=5/mvcc=0, 
> row010/cf1:ip/1389104461530/Put/vlen=11/mvcc=0, 
> row010/cf1:port-80/1389104461530/Put/vlen=5/mvcc=0, 
> row010/cf2:dummy1/1389104461530/Put/vlen=6/mvcc=0, 
> row010/cf2:dummy2/1389104461530/Put/vlen=6/mvcc=0}
> {code}
> 2. Then I use following code to try to filter out only one row I want
> {code:title=FilterList codes}
> //...
> fl = new FilterList(Operator.MUST_PASS_ALL);
> filter =
>     new SingleColumnValueFilter(CF_1_NAME_B, Bytes.toBytes("ip"), 
> CompareOp.EQUAL,
>         Bytes.toBytes("127.0.0.80"));
> fl.addFilter(filter);
> filter = new QualifierFilter(CompareOp.EQUAL, new 
> BinaryComparator(Bytes.toBytes("port-80")));
> fl.addFilter(filter);
> scan.setFilter(fl);
> rs = table.getScanner(scan);
> for (Result r : rs) {
>   //...
> }
> {code}
> 3. I get the right and only one row returned, but not get whole data 
> (_KeyValue_s) of this row.
> {code:title=Results not expected}
> r=keyvalues={row008/cf1:port-80/1389104629712/Put/vlen=5/mvcc=0}
> {code}
> Actually, I expect following data would return, whole record data, two column 
> families and five column qualifiers.
> {code:tittle=Results I expected}
> r=keyvalues={row008/cf1:dummy/1389104461530/Put/vlen=5/mvcc=0, 
> row008/cf1:ip/1389104461530/Put/vlen=10/mvcc=0, 
> row008/cf1:port-80/1389104461530/Put/vlen=5/mvcc=0, 
> row008/cf2:dummy1/1389104461530/Put/vlen=6/mvcc=0, 
> row008/cf2:dummy2/1389104461530/Put/vlen=6/mvcc=0}
> {code}
> h4.Test for Operator.MUST_PASS_ONE
> then I test the same code except to change the _Operator.MUST_PASS_ALL_ to 
> _Operator.MUST_PASS_ONE_, and things getting more worse...
> {code:title=Result not expected}
> r=keyvalues={row001/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0}
> r=keyvalues={row002/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0}
> r=keyvalues={row003/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0}
> r=keyvalues={row004/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0}
> r=keyvalues={row005/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0}
> r=keyvalues={row006/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0}
> r=keyvalues={row007/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0, 
> row007/cf1:ip/1389113376745/Put/vlen=10/mvcc=0, 
> row007/cf1:port-70/1389113376745/Put/vlen=5/mvcc=0, 
> row007/cf2:dummy1/1389113376745/Put/vlen=6/mvcc=0, 
> row007/cf2:dummy2/1389113376745/Put/vlen=6/mvcc=0}
> r=keyvalues={row008/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0, 
> row008/cf1:ip/1389113376745/Put/vlen=10/mvcc=0, 
> row008/cf1:port-80/1389113376745/Put/vlen=5/mvcc=0, 
> row008/cf2:dummy1/1389113376745/Put/vlen=6/mvcc=0, 
> row008/cf2:dummy2/1389113376745/Put/vlen=6/mvcc=0}
> r=keyvalues={row009/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0, 
> row009/cf1:port-80/1389113376745/Put/vlen=5/mvcc=0}
> r=keyvalues={row010/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0, 
> row010/cf1:port-80/1389113376745/Put/vlen=5/mvcc=0}
> {code}
> But I only expected following two row got returned
> {code:title=Results I expected}
> r=keyvalues={row007/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0, 
> row007/cf1:ip/1389113376745/Put/vlen=10/mvcc=0, 
> row007/cf1:port-70/1389113376745/Put/vlen=5/mvcc=0, 
> row007/cf2:dummy1/1389113376745/Put/vlen=6/mvcc=0, 
> row007/cf2:dummy2/1389113376745/Put/vlen=6/mvcc=0}
> r=keyvalues={row008/cf1:dummy/1389113376745/Put/vlen=5/mvcc=0, 
> row008/cf1:ip/1389113376745/Put/vlen=10/mvcc=0, 
> row008/cf1:port-80/1389113376745/Put/vlen=5/mvcc=0, 
> row008/cf2:dummy1/1389113376745/Put/vlen=6/mvcc=0, 
> row008/cf2:dummy2/1389113376745/Put/vlen=6/mvcc=0}
> {code}
> Test code attached to demo this issue.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Reply via email to