Hi Lars:

Thanks for the reply.
I need to understand if I misunderstood the perceived inefficiency because
it seems you don't think quite the same.

Let say, as an example, we have 1 row with 2 columns (col-1 and col-2) in a
table and each column has 1000 versions. Using the following code (the code
might have errors and don't compile):
/**
 * This is very simple use case of a ColumnPrefixFilter.
 * In fact all other filters that make use of filterKeyValue will see
similar
 * performance problems that I have concerned with when the number of
 * versions per column could be huge.

Filter filter = new ColumnPrefixFilter(Bytes.toBytes("col-2"));
Scan scan = new Scan();
scan.setFilter(filter);
ResultScanner scanner = table.getScanner(scan);
for (Result result : scanner) {
    for (KeyValue kv : result.raw()) {
        System.out.println("KV: " + kv + ", Value: " +
        Bytes.toString(kv.getValue()));
    }
}
scanner.close();
*/

Implicitly, the number of version per column that is going to return is 1
(the latest version). User might expect that only 2 comparisons for column
prefix are needed (1 for col-1 and 1 for col-2) but in fact, it processes
the filterKeyValue method in ColumnPrefixFilter 1000 times (1 for col-1 and
1000 for col-2) for col-2 (1 per version) because all versions of the
column have the same prefix for obvious reason. For col-1, it will skip
using SEEK_NEXT_USING_HINT which should skip the 99 versions of col-1.

In summary, the 1000 comparisons (5000 byte comparisons) for the column
prefix "col-2" is wasted because only 1 version is returned to user. Also,
I believe this inefficiency is hidden from the user code but it affects all
filters that use filterKeyValue as the main execution for filtering KVs. Do
we have a case to improve HBase to handle this inefficiency? :) It seems
valid unless you prove otherwise.

Best Regards,

Jerry



On Tue, Aug 28, 2012 at 12:54 AM, lars hofhansl <lhofha...@yahoo.com> wrote:

> First off regarding "inefficiency"... If version counting would happen
> first and then filter were executed we'd have folks "complaining" about
> inefficiencies as well:
> ("Why does the code have to go through the versioning stuff when my filter
> filters the row/column/version anyway?")  ;-)
>
>
> For your problem, you want to make use of "seek hints"...
>
> In addition to INCLUDE you can return NEXT_COL, NEXT_ROW, or even
> SEEK_NEXT_USING_HINT from Filter.filterKeyValue(...).
>
> That way the scanning framework will know to skip ahead to the next
> column, row, or a KV of your choosing. (see Filter.filterKeyValue and
> Filter.getNextKeyHint).
>
> (as an aside, it would probably be nice if Filters also had
> INCLUDE_AND_NEXT_COL, INCLUDE_AND_NEXT_ROW, internally used by StoreScanner)
>
> Have a look at ColumnPrefixFilter as an example.
> I also wrote a short post here:
> http://hadoop-hbase.blogspot.com/2012/01/filters-in-hbase-or-intra-row-scanning.html
>
> Does that help?
>
> -- Lars
>
>
> ----- Original Message -----
> From: Jerry Lam <chiling...@gmail.com>
> To: "user@hbase.apache.org" <user@hbase.apache.org>
> Cc: "user@hbase.apache.org" <user@hbase.apache.org>
> Sent: Monday, August 27, 2012 5:59 PM
> Subject: Re: setTimeRange and setMaxVersions seem to be inefficient
>
> Hi Lars:
>
> Thanks for confirming the inefficiency of the implementation for this
> case. For my case, a column can have more than 10K versions, I need a quick
> way to stop the scan from digging the column once there is a match
> (ReturnCode.INCLUDE). It would be nice to have a ReturnCode that can notify
> the framework to stop and go to next column once the number of versions
> specify in setMaxVersions is met.
>
> For now, I guess I have to hack it in the custom filter (I.e. I keep the
> count myself)? If you have a better way to achieve this, please share :)
>
> Best Regards,
>
> Jerry
>
> Sent from my iPad (sorry for spelling mistakes)
>
> On 2012-08-27, at 20:11, lars hofhansl <lhofha...@yahoo.com> wrote:
>
> > Currently filters are evaluated before we do version counting.
> >
> > Here's a comment from ScanQueryMatcher.java:
> >     /**
> >      * 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.
> >      */
> >
> >
> > So this is by design. (Doesn't mean it's correct or desirable, though.)
> >
> > -- Lars
> >
> >
> > ----- Original Message -----
> > From: Jerry Lam <chiling...@gmail.com>
> > To: user <user@hbase.apache.org>
> > Cc:
> > Sent: Monday, August 27, 2012 2:40 PM
> > Subject: setTimeRange and setMaxVersions seem to be inefficient
> >
> > Hi HBase community:
> >
> > I tried to use setTimeRange and setMaxVersions to limit the number of KVs
> > return per column. The behaviour is as I would expect that is
> > setTimeRange(0, T + 1) and setMaxVersions(1) will give me ONE version of
> KV
> > with timestamp that is less than or equal to T.
> > However, I noticed that all versions of the KeyValue for a particular
> > column are processed through a custom filter I implemented even though I
> > specify setMaxVersions(1) and setTimeRange(0, T+1). I expected that if
> ONE
> > KV of a particular column has ReturnCode.INCLUDE, the framework will jump
> > to the next COL instead of iterating through all versions of the column.
> >
> > Can someone confirm me if this is the expected behaviour (iterating
> through
> > all versions of a column before setMaxVersions take effect)? If this is
> an
> > expected behaviour, what is your recommendation to speed this up?
> >
> > Best Regards,
> >
> > Jerry
> >
>
>

Reply via email to