Monday, January 23, 2012

HBase intra row scanning

By Lars Hofhansl

Updated (again) Wednesday, January 25th, 2012.

As I painfully worked through HBASE-5229 I realized that HBase already has all the building blocks needed for complex (local) transactions.

What's important here is that (see my introduction to HBase):
  1. HBase ensures atomicity for operations for the same row key
  2. HBase keys have internal structure: (row-key, column family, column, ...)
The missing piece was ColumnRangeFilter. With this filter it is possible to retrieve all columns whose identifier starts with "abc", or all columns whose identifier sorts > "test". For example:

// all columns whose identifier starts with "abc"
Filter f = new ColumnRangeFilter(Bytes.toBytes("abc"), true,
                                 Bytes.toBytes("abd"), false);

// all columns whose identifier sorts after "test"
Filter f = new ColumnRangeFilter(Bytes.toBytes("test"), true,
                                 null, true);


So this allows to search (scan) inside a row by column identifier just  as HBase allows searching by row key.

A client application can exploit this to achieve transactions by grouping all entities that can participate in the same transaction into a single row (and single column family).
Then using prefixes of the column identifiers can be used to define rows inside that group. Basically the search criteria for keys was moved one level down to the column identifier.

Say we wanted to implement a store with transactional tables that contain rows and columns. One way to doing this with HBase as follows:
  • the HBase row-key/column-family maps to a "table"
  • a prefix of the HBase column identifier maps to a "row"
  • the rest of the HBase column identifier identifies the "column"
This is in fact similar to what Google's Megastore (pdf) does.

This leads to potentially wide HBase rows with many columns. The missing piece is allowing a Scan to efficiently retrieve a slice of a wide row.

This where ColumnRangeFilter comes into play. This filter seeks efficiently into the row by seeking ahead to the first HBase block that contains the first KeyValue (or cell) for that column.

Let's model a table "pets" this way. And let's say a pet has a name and a species. The HBase key for entries would look like this:
(table, CF1, rowA|column1) -> value for column1 in rowA
The code would look something like this:
(apologies for the initial incorrect code that I had posted here)

HTable t = ...;
Scan s = ...;
s.setStartRow("pets");
s.setStopRow("pets");
// get all columns for my pet "fluffy".
Filter f = new ColumnRangeFilter(Bytes.toBytes("fluffy"), true,
                                 Bytes.toBytes("fluffz"), false);
s.setFilter(f);
s.setBatch(20); // avoid getting all columns for the HBase row
ResultScanner rs = t.getScanner(s);
for (Result r = rs.next(); r != null; r = rs.next()) {
  // r will now have all HBase columns that start with "fluffy",
  // which would represent a single row
  for (KeyValue kv : r.raw()) {
    // each kv represent - the latest version of - a column
  }
}

The downside of this is that HBase achieves atomicity by collocating all cells with the same row-key, so it has to be hosted by a single region server.

5 comments:

  1. Hi Lars,

    Thanks for the gr8 post. I had a doubt: Does HBase store the cells indexed by the column-qualifiers or is it a list ? If it is a list then will Column-Range-Filter and Column-Prefix-Filter result into a scan of all the qualifiers in that column-family for that row-key ?

    Thanks
    Kisalay

    ReplyDelete
  2. Just discovered your blog: great stuff!

    ReplyDelete
  3. @Kisalay: The column qualifiers are sorted within their row (and column family). That is why the ColumnRangeFilter can seek ahead efficiently.

    @Bruno: Thanks. I am glad you find it useful. More to come.

    ReplyDelete
  4. Hi Lars,

    Great post, it really changes the way I think when it comes to schema design. quick question - does ColumnRangeFilter work without a rowkey. eg. I want query all the columns starting with "abc" and ending with "abd" for all the rows. if it does, I think the efficiency will be just like scan rowkey, right?

    ReplyDelete
  5. Hi Shengjie,

    ColumnRangeFilter does work without selected in the Scan object.
    The point here, though, to avoid scans over many KeyValue (columns). Imagine a single row with 1 million columns. Without ColumnRangeFilter or ColumnPrefixFilter you would have to retrieve all column even if you only care about a subset of them.

    Similarly, if you use a ColumnRangeFilter without limiting the row keys HBase will scan over all rows in the table. Within these rows it will efficiently only return the columns you are interested in. So it depends on what you want to achieve.

    ReplyDelete