hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sudarshan Kadambi (BLOOMBERG/ 731 LEXIN)" <skada...@bloomberg.net>
Subject Re: PrefixFilter
Date Mon, 12 Aug 2013 20:55:03 GMT
I'm willing to be told I'm completely wrong here, but it seems like the prefix filter should
be capable of using the same mechanism used in a row-key lookup or a scan with a start and
stop row.

If HBase were to be like a hash table with no notion of sorted-ness, I can understand a partial-key
lookup requiring something akin to a full-table scan. But given that HBase orders records
by row-key, a prefix lookup should be able to do a binary search over the index? 

----- Original Message -----
From: yuzhihong@gmail.com
To: user@hbase.apache.org
Cc: Sudarshan Kadambi (BLOOMBERG/ 731 LEXIN)
At: Aug 12 2013 16:06:58

Adding back user@

bq. does it jump directly to Prefix3

I don't think so.

Are your prefixes of fixed length ?
If so, take a look at FuzzyRowFilter.


On Mon, Aug 12, 2013 at 11:33 AM, Sudarshan Kadambi (BLOOMBERG/ 731 LEXIN) <skadambi@bloomberg.net>

Ted: Thanks for looking that up.

If I have rows with the following keys in my table (let's say table has only 1 region):

and if I specify a prefix filter with Prefix3, does it jump directly to Prefix3, or does it
read in both Prefix1/Suffix and Prefix2/Suffix and discard them before returning Prefix3/Suffix
and Prefix3/Suffix2.

Using the prefix filter is much slower than a scan with start row/end row and I'm trying to
understand why. Thanks!

----- Original Message -----
From: yuzhihong@gmail.com
To: Sudarshan Kadambi (BLOOMBERG/ 731 LEXIN), user@hbase.apache.org
At: Aug 12 2013 14:08:17

In filterAllRemaining() method:
  public boolean filterAllRemaining() {
    return passedPrefix;
In filterRowKey():
    // if they are equal, return false => pass row
    // else return true, filter row
    // if we are passed the prefix, set flag
    int cmp = Bytes.compareTo(buffer, offset, this.prefix.length, this.prefix, 0,
    if(cmp > 0) {
      passedPrefix = true;
So once the prefix has passed, the remaining rows would be skipped.
On Mon, Aug 12, 2013 at 11:01 AM, Sudarshan Kadambi (BLOOMBERG/ 731 LEXIN) <skadambi@bloomberg.net>
Anyone know if the prefix filter[1] does a full table scan?

1 - http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/PrefixFilter.html

  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message