On Mon, Nov 2, 2009 at 11:02 AM, bmdevelopment <[email protected]>wrote:

> Hello,
> I am back to where I am going to need the functionality I have described
> below - basically a SCVF that works on all versions. I was wondering if I
> may go ahead and open an issue for this and begin to contribute some code.
> Granted, I will have to do some digging to get my head around the filtering
> at the moment. If that is fine, then I will go ahead and do so and post in
> JIRA once I have something working.
>


Please do.  Let us know if we can help out.  Make sure you get latest code.
Some fixes have gone in for filters recently.

Yours,
St.Ack



> Thanks
> J
>
> Bluemetrix Development wrote:
>
>> Hi, Thanks for the response.
>> Yes, ValueFilter does compare against all versions. Thanks for
>> pointing this out. I'm getting closer. :)
>>
>> Basically, I am wanting to do the following:
>> Get all values (for all versions) of a row where a specific value (for
>> any version), specified by family:qualifier, matches a string.
>>
>> The problems I am having now are that
>> 1.) I can specify a string to match with ValueFilter, but now only
>> that value is returned with the row. I want all values in that row
>> returned.
>> 2.) I cannot specify which family:qualifier to match against for a
>> past version. This causes problems if the value I am looking for is,
>> by chance, in a column other than the one I want to search, therefore
>> returning a row that I do not really want. SCVF solves this, but does
>> not work for past versions.
>>
>> I've started to work around this by filtering on the client side after
>> the results are returned, but of course it would be nice not to return
>> rows that will not be used in the final data set.
>> Also, there are probably ways to work around this by re-structuring my
>> table, but that could lead to many different tables per query type.
>>
>> Thanks again.
>> J
>>
>> On Thu, Oct 1, 2009 at 2:50 PM, Jonathan Gray <[email protected]> wrote:
>>
>>> That is the behavior for SCVF.  The other filters generally don't pay
>>> attention to versions, but SCVF is special because it makes the decision
>>> once it trips over the sought after column (the first/most recent version
>>> of it).
>>>
>>> What exactly are you trying to do?  Could you use ValueFilter instead?
>>>
>>> JG
>>>
>>> On Thu, October 1, 2009 10:06 am, Bluemetrix Development wrote:
>>>
>>>> Hi, Thanks very much.
>>>> I applied the patch and the RowFilter seem to be initially working.
>>>>
>>>>
>>>> But, I have seem to come across another problem now.
>>>> Filters only compare against the most recent version's values. Is this
>>>> correct?
>>>>
>>>> For example, let's say I have a row with 3 versions:
>>>> hbase(main):004:0> get 'Data', 'abc', {COLUMNS=>'x:y', VERSIONS=>3}
>>>> timestamp=124380366942072, value=xxx timestamp=124380364890514,
>>>> value=yyy
>>>> timestamp=124380364491516, value=zzz
>>>>
>>>> Now, I want to be able to use a SingleColumnValueFilter on 'x:y' to
>>>> find the value 'zzz'. It seems I can only test against the latest
>>>> version
>>>> 'xxx'.
>>>> Sample code:
>>>> Filter f = new SingleColumnValueFilter(Bytes.toBytes("x"),
>>>> Bytes.toBytes("y"), CompareOp.EQUAL, new SubstringComparator("zzz"));
>>>> This will not find anything.
>>>>
>>>>
>>>> Is this the intended behavior or am I doing something wrong?
>>>> Thanks again
>>>> J
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On Wed, Sep 30, 2009 at 6:00 PM, Daniel Ploeg <[email protected]> wrote:
>>>>
>>>>  Looks like your running into the issue described in this jira:
>>>>>
>>>>>
>>>>> https://issues.apache.org/jira/browse/HBASE-1828
>>>>>
>>>>>
>>>>> On Thu, Oct 1, 2009 at 5:52 AM, Bluemetrix Development <
>>>>> [email protected]> wrote:
>>>>>
>>>>>  Hello,
>>>>>> I am trying to use a RowFilter and continue to get the following
>>>>>> error:
>>>>>>
>>>>>>
>>>>>> Exception in thread "main"
>>>>>> org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
>>>>>> contact region server 127.0.0.1:54441 for region
>>>>>> TableData,,1254329413425, row '', but failed after 10 attempts.
>>>>>> Exceptions:
>>>>>> java.io.IOException: Call to /127.0.0.1:54441 failed on local
>>>>>> exception: java.io.EOFException
>>>>>> java.io.IOException: Call to /127.0.0.1:54441 failed on local
>>>>>> exception: java.io.EOFException
>>>>>> java.io.IOException: Call to /127.0.0.1:54441 failed on local
>>>>>> exception: java.io.EOFException
>>>>>> ...
>>>>>>
>>>>>>
>>>>>> Here is the code:
>>>>>>
>>>>>>
>>>>>>       HBaseConfiguration conf = new HBaseConfiguration();
>>>>>>       HTable table = new HTable(conf, tableName);
>>>>>>       Scan scan = new Scan();
>>>>>>       scan.setMaxVersions();
>>>>>>       scan.setTimeRange(timeStart, timeEnd);
>>>>>>       RowFilter rf = new RowFilter(CompareOp.EQUAL, new
>>>>>> SubstringComparator("abc"));
>>>>>>       scan.setFilter(rf);
>>>>>>       ResultScanner scanner = table.getScanner(scan);
>>>>>>
>>>>>>
>>>>>> I've never came across this before until I started trying to use
>>>>>> RowFilter. And if I comment out the setFilter function, there is no
>>>>>> longer a problem. Actually, I get the exact same error for any type of
>>>>>>  Filter I attempt to use. I'm using 0.20 on a pseudo-dist setup -
>>>>>> standard config as in the Getting Started docs. Any ideas on this? Am
>>>>>> I
>>>>>> using the Filter correctly?
>>>>>>
>>>>>> Thanks
>>>>>> J
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>

Reply via email to