<link xlink:href="http://www.hbase.org"> Apache HBase Reference Guide </link> 2012Apache Software Foundation This is the official reference guide of Apache HBase, a distributed, versioned, column-oriented database built on top of Apache Hadoop and Apache ZooKeeper. Data Model In short, applications store data into an HBase table. Tables are made of rows and columns. All columns in HBase belong to a particular column family. Table cells -- the intersection of row and column coordinates -- are versioned. A cell’s content is an uninterpreted array of bytes. Table row keys are also byte arrays so almost anything can serve as a row key from strings to binary representations of longs or even serialized data structures. Rows in HBase tables are sorted by row key. The sort is byte-ordered. All table accesses are via the table row key -- its primary key.
Conceptual View The following example is a slightly modified form of the one on page 2 of the BigTable paper. There is a table called webtable that contains two column families named contents and anchor. In this example, anchor contains two columns (anchor:cssnsi.com, anchor:my.look.ca) and contents contains one column (contents:html). Column Names By convention, a column name is made of its column family prefix and a qualifier. For example, the column contents:html is of the column family contents The colon character (:) delimits the column family from the column family qualifier. Table <varname>webtable</varname> Row KeyTime StampColumnFamily contentsColumnFamily anchor"com.cnn.www"t9anchor:cnnsi.com = "CNN""com.cnn.www"t8anchor:my.look.ca = "CNN.com""com.cnn.www"t6contents:html = "<html>...""com.cnn.www"t5contents:html = "<html>...""com.cnn.www"t3contents:html = "<html>..."
Physical View Although at a conceptual level tables may be viewed as a sparse set of rows. Physically they are stored on a per-column family basis. New columns (i.e., columnfamily:column) can be added to any column family without pre-announcing them. ColumnFamily <varname>anchor</varname> Row KeyTime StampColumn Family anchor"com.cnn.www"t9anchor:cnnsi.com = "CNN""com.cnn.www"t8anchor:my.look.ca = "CNN.com"
ColumnFamily <varname>contents</varname> Row KeyTime StampColumnFamily "contents:""com.cnn.www"t6contents:html = "<html>...""com.cnn.www"t5contents:html = "<html>...""com.cnn.www"t3contents:html = "<html>..."
It is important to note in the diagram above that the empty cells shown in the conceptual view are not stored since they need not be in a column-oriented storage format. Thus a request for the value of the contents:html column at time stamp t8 would return no value. Similarly, a request for an anchor:my.look.ca value at time stamp t9 would return no value. However, if no timestamp is supplied, the most recent value for a particular column would be returned and would also be the first one found since timestamps are stored in descending order. Thus a request for the values of all columns in the row com.cnn.www if no timestamp is specified would be: the value of contents:html from time stamp t6, the value of anchor:cnnsi.com from time stamp t9, the value of anchor:my.look.ca from time stamp t8.
For more information about the internals of how HBase stores data, see .
Table Tables are declared up front at schema definition time.
Row Row keys are uninterrpreted bytes. Rows are lexicographically sorted with the lowest order appearing first in a table. The empty byte array is used to denote both the start and end of a tables' namespace.
Column Family<indexterm><primary>Column Family</primary></indexterm> Columns in HBase are grouped into column families. All column members of a column family have the same prefix. For example, the columns courses:history and courses:math are both members of the courses column family. The colon character (:) delimits the column family from the column family qualifierColumn Family Qualifier. The column family prefix must be composed of printable characters. The qualifying tail, the column family qualifier, can be made of any arbitrary bytes. Column families must be declared up front at schema definition time whereas columns do not need to be defined at schema time but can be conjured on the fly while the table is up an running. Physically, all column family members are stored together on the filesystem. Because tunings and storage specifications are done at the column family level, it is advised that all column family members have the same general access pattern and size characteristics.
Cells<indexterm><primary>Cells</primary></indexterm> A {row, column, version} tuple exactly specifies a cell in HBase. Cell content is uninterrpreted bytes
Data Model Operations The four primary data model operations are Get, Put, Scan, and Delete. Operations are applied via HTable instances.
Get Get returns attributes for a specified row. Gets are executed via HTable.get.
Put Put either adds new rows to a table (if the key is new) or can update existing rows (if the key already exists). Puts are executed via HTable.put (writeBuffer) or HTable.batch (non-writeBuffer).
Scans Scan allow iteration over multiple rows for specified attributes. The following is an example of a on an HTable table instance. Assume that a table is populated with rows with keys "row1", "row2", "row3", and then another set of rows with the keys "abc1", "abc2", and "abc3". The following example shows how startRow and stopRow can be applied to a Scan instance to return the rows beginning with "row". HTable htable = ... // instantiate HTable Scan scan = new Scan(); scan.addColumn(Bytes.toBytes("cf"),Bytes.toBytes("attr")); scan.setStartRow( Bytes.toBytes("row")); // start key is inclusive scan.setStopRow( Bytes.toBytes("row" + (char)0)); // stop key is exclusive ResultScanner rs = htable.getScanner(scan); try { for (Result r = rs.next(); r != null; r = rs.next()) { // process result... } finally { rs.close(); // always close the ResultScanner! }
Delete Delete removes a row from a table. Deletes are executed via HTable.delete. HBase does not modify data in place, and so deletes are handled by creating new markers called tombstones. These tombstones, along with the dead values, are cleaned up on major compactions. See for more information on deleting versions of columns, and see for more information on compactions.
Versions<indexterm><primary>Versions</primary></indexterm> A {row, column, version} tuple exactly specifies a cell in HBase. Its possible to have an unbounded number of cells where the row and column are the same but the cell address differs only in its version dimension. While rows and column keys are expressed as bytes, the version is specified using a long integer. Typically this long contains time instances such as those returned by java.util.Date.getTime() or System.currentTimeMillis(), that is: the difference, measured in milliseconds, between the current time and midnight, January 1, 1970 UTC. The HBase version dimension is stored in decreasing order, so that when reading from a store file, the most recent values are found first. There is a lot of confusion over the semantics of cell versions, in HBase. In particular, a couple questions that often come up are: If multiple writes to a cell have the same version, are all versions maintained or just the last? Currently, only the last written is fetchable. Is it OK to write cells in a non-increasing version order? Yes Below we describe how the version dimension in HBase currently works See HBASE-2406 for discussion of HBase versions. Bending time in HBase makes for a good read on the version, or time, dimension in HBase. It has more detail on versioning than is provided here. As of this writing, the limiitation Overwriting values at existing timestamps mentioned in the article no longer holds in HBase. This section is basically a synopsis of this article by Bruno Dumon. .
Versions and HBase Operations In this section we look at the behavior of the version dimension for each of the core HBase operations.
Get/Scan Gets are implemented on top of Scans. The below discussion of Get applies equally to Scans. By default, i.e. if you specify no explicit version, when doing a get, the cell whose version has the largest value is returned (which may or may not be the latest one written, see later). The default behavior can be modified in the following ways: to return more than one version, see Get.setMaxVersions() to return versions other than the latest, see Get.setTimeRange() To retrieve the latest version that is less than or equal to a given value, thus giving the 'latest' state of the record at a certain point in time, just use a range from 0 to the desired version and set the max versions to 1.
Default Get Example The following Get will only retrieve the current version of the row Get get = new Get(Bytes.toBytes("row1")); Result r = htable.get(get); byte[] b = r.getValue(Bytes.toBytes("cf"), Bytes.toBytes("attr")); // returns current version of value
Versioned Get Example The following Get will return the last 3 versions of the row. Get get = new Get(Bytes.toBytes("row1")); get.setMaxVersions(3); // will return last 3 versions of row Result r = htable.get(get); byte[] b = r.getValue(Bytes.toBytes("cf"), Bytes.toBytes("attr")); // returns current version of value List<KeyValue> kv = r.getColumn(Bytes.toBytes("cf"), Bytes.toBytes("attr")); // returns all versions of this column
Put Doing a put always creates a new version of a cell, at a certain timestamp. By default the system uses the server's currentTimeMillis, but you can specify the version (= the long integer) yourself, on a per-column level. This means you could assign a time in the past or the future, or use the long value for non-time purposes. To overwrite an existing value, do a put at exactly the same row, column, and version as that of the cell you would overshadow.
Implicit Version Example The following Put will be implicitly versioned by HBase with the current time. Put put = new Put(Bytes.toBytes(row)); put.add(Bytes.toBytes("cf"), Bytes.toBytes("attr1"), Bytes.toBytes( data)); htable.put(put);
Explicit Version Example The following Put has the version timestamp explicitly set. Put put = new Put( Bytes.toBytes(row)); long explicitTimeInMs = 555; // just an example put.add(Bytes.toBytes("cf"), Bytes.toBytes("attr1"), explicitTimeInMs, Bytes.toBytes(data)); htable.put(put); Caution: the version timestamp is internally by HBase for things like time-to-live calculations. It's usually best to avoid setting this timestamp yourself. Prefer using a separate timestamp attribute of the row, or have the timestamp a part of the rowkey, or both.
Delete There are three different types of internal delete markers See Lars Hofhansl's blog for discussion of his attempt adding another, Scanning in HBase: Prefix Delete Marker: Delete: for a specific version of a column. Delete column: for all versions of a column. Delete family: for all columns of a particular ColumnFamily When deleting an entire row, HBase will internally create a tombstone for each ColumnFamily (i.e., not each individual column). Deletes work by creating tombstone markers. For example, let's suppose we want to delete a row. For this you can specify a version, or else by default the currentTimeMillis is used. What this means is delete all cells where the version is less than or equal to this version. HBase never modifies data in place, so for example a delete will not immediately delete (or mark as deleted) the entries in the storage file that correspond to the delete condition. Rather, a so-called tombstone is written, which will mask the deleted values When HBase does a major compaction, the tombstones are processed to actually remove the dead values, together with the tombstones themselves. . If the version you specified when deleting a row is larger than the version of any value in the row, then you can consider the complete row to be deleted. For an informative discussion on how deletes and versioning interact, see the thread Put w/ timestamp -> Deleteall -> Put w/ timestamp fails up on the user mailing list. Also see for more information on the internal KeyValue format.
Current Limitations There are still some bugs (or at least 'undecided behavior') with the version dimension that will be addressed by later HBase releases.
Deletes mask Puts Deletes mask puts, even puts that happened after the delete was entered HBASE-2256 . Remember that a delete writes a tombstone, which only disappears after then next major compaction has run. Suppose you do a delete of everything <= T. After this you do a new put with a timestamp <= T. This put, even if it happened after the delete, will be masked by the delete tombstone. Performing the put will not fail, but when you do a get you will notice the put did have no effect. It will start working again after the major compaction has run. These issues should not be a problem if you use always-increasing versions for new puts to a row. But they can occur even if you do not care about time: just do delete and put immediately after each other, and there is some chance they happen within the same millisecond.
Major compactions change query results ...create three cell versions at t1, t2 and t3, with a maximum-versions setting of 2. So when getting all versions, only the values at t2 and t3 will be returned. But if you delete the version at t2 or t3, the one at t1 will appear again. Obviously, once a major compaction has run, such behavior will not be the case anymore... See Garbage Collection in Bending time in HBase
Sort Order All data model operations HBase return data in sorted order. First by row, then by ColumnFamily, followed by column qualifier, and finally timestamp (sorted in reverse, so newest records are returned first).
Column Metadata There is no store of column metadata outside of the internal KeyValue instances for a ColumnFamily. Thus, while HBase can support not only a wide number of columns per row, but a heterogenous set of columns between rows as well, it is your responsibility to keep track of the column names. The only way to get a complete set of columns that exist for a ColumnFamily is to process all the rows. For more information about how HBase stores data internally, see .
Joins Whether HBase supports joins is a common question on the dist-list, and there is a simple answer: it doesn't, at not least in the way that RDBMS' support them (e.g., with equi-joins or outer-joins in SQL). As has been illustrated in this chapter, the read data model operations in HBase are Get and Scan. However, that doesn't mean that equivalent join functionality can't be supported in your application, but you have to do it yourself. The two primary strategies are either denormalizing the data upon writing to HBase, or to have lookup tables and do the join between HBase tables in your application or MapReduce code (and as RDBMS' demonstrate, there are several strategies for this depending on the size of the tables, e.g., nested loops vs. hash-joins). So which is the best approach? It depends on what you are trying to do, and as such there isn't a single answer that works for every use case.
HBase and Schema Design A good general introduction on the strength and weaknesses modelling on the various non-rdbms datastores is Ian Varleys' Master thesis, No Relation: The Mixed Blessings of Non-Relational Databases. Recommended. Also, read for how HBase stores data internally.
Schema Creation HBase schemas can be created or updated with or by using HBaseAdmin in the Java API. Tables must be disabled when making ColumnFamily modifications, for example.. Configuration config = HBaseConfiguration.create(); HBaseAdmin admin = new HBaseAdmin(conf); String table = "myTable"; admin.disableTable(table); HColumnDescriptor cf1 = ...; admin.addColumn(table, cf1); // adding new ColumnFamily HColumnDescriptor cf2 = ...; admin.modifyColumn(table, cf2); // modifying existing ColumnFamily admin.enableTable(table); See for more information about configuring client connections. Note: online schema changes are supported in the 0.92.x codebase, but the 0.90.x codebase requires the table to be disabled.
Schema Updates When changes are made to either Tables or ColumnFamilies (e.g., region size, block size), these changes take effect the next time there is a major compaction and the StoreFiles get re-written. See for more information on StoreFiles.
On the number of column families HBase currently does not do well with anything above two or three column families so keep the number of column families in your schema low. Currently, flushing and compactions are done on a per Region basis so if one column family is carrying the bulk of the data bringing on flushes, the adjacent families will also be flushed though the amount of data they carry is small. When many column families the flushing and compaction interaction can make for a bunch of needless i/o loading (To be addressed by changing flushing and compaction to work on a per column family basis). For more information on compactions, see . Try to make do with one column family if you can in your schemas. Only introduce a second and third column family in the case where data access is usually column scoped; i.e. you query one column family or the other but usually not both at the one time.
Cardinality of ColumnFamilies Where multiple ColumnFamilies exist in a single table, be aware of the cardinality (i.e., number of rows). If ColumnFamilyA has 1 million rows and ColumnFamilyB has 1 billion rows, ColumnFamilyA's data will likely be spread across many, many regions (and RegionServers). This makes mass scans for ColumnFamilyA less efficient.
Rowkey Design
Monotonically Increasing Row Keys/Timeseries Data In the HBase chapter of Tom White's book Hadoop: The Definitive Guide (O'Reilly) there is a an optimization note on watching out for a phenomenon where an import process walks in lock-step with all clients in concert pounding one of the table's regions (and thus, a single node), then moving onto the next region, etc. With monotonically increasing row-keys (i.e., using a timestamp), this will happen. See this comic by IKai Lan on why monotonically increasing row keys are problematic in BigTable-like datastores: monotonically increasing values are bad. The pile-up on a single region brought on by monotonically increasing keys can be mitigated by randomizing the input records to not be in sorted order, but in general its best to avoid using a timestamp or a sequence (e.g. 1, 2, 3) as the row-key. If you do need to upload time series data into HBase, you should study OpenTSDB as a successful example. It has a page describing the schema it uses in HBase. The key format in OpenTSDB is effectively [metric_type][event_timestamp], which would appear at first glance to contradict the previous advice about not using a timestamp as the key. However, the difference is that the timestamp is not in the lead position of the key, and the design assumption is that there are dozens or hundreds (or more) of different metric types. Thus, even with a continual stream of input data with a mix of metric types, the Puts are distributed across various points of regions in the table.
Try to minimize row and column sizes Or why are my StoreFile indices large? In HBase, values are always freighted with their coordinates; as a cell value passes through the system, it'll be accompanied by its row, column name, and timestamp - always. If your rows and column names are large, especially compared to the size of the cell value, then you may run up against some interesting scenarios. One such is the case described by Marc Limotte at the tail of HBASE-3551 (recommended!). Therein, the indices that are kept on HBase storefiles () to facilitate random access may end up occupyng large chunks of the HBase allotted RAM because the cell value coordinates are large. Mark in the above cited comment suggests upping the block size so entries in the store file index happen at a larger interval or modify the table schema so it makes for smaller rows and column names. Compression will also make for larger indices. See the thread a question storefileIndexSize up on the user mailing list. Most of the time small inefficiencies don't matter all that much. Unfortunately, this is a case where they do. Whatever patterns are selected for ColumnFamilies, attributes, and rowkeys they could be repeated several billion times in your data. See for more information on HBase stores data internally to see why this is important.
Column Families Try to keep the ColumnFamily names as small as possible, preferably one character (e.g. "d" for data/default). See for more information on HBase stores data internally to see why this is important.
Attributes Although verbose attribute names (e.g., "myVeryImportantAttribute") are easier to read, prefer shorter attribute names (e.g., "via") to store in HBase. See for more information on HBase stores data internally to see why this is important.
Rowkey Length Keep them as short as is reasonable such that they can still be useful for required data access (e.g., Get vs. Scan). A short key that is useless for data access is not better than a longer key with better get/scan properties. Expect tradeoffs when designing rowkeys.
Byte Patterns A long is 8 bytes. You can store an unsigned number up to 18,446,744,073,709,551,615 in those eight bytes. If you stored this number as a String -- presuming a byte per character -- you need nearly 3x the bytes. Not convinced? Below is some sample code that you can run on your own. // long // long l = 1234567890L; byte[] lb = Bytes.toBytes(l); System.out.println("long bytes length: " + lb.length); // returns 8 String s = "" + l; byte[] sb = Bytes.toBytes(s); System.out.println("long as string length: " + sb.length); // returns 10 // hash // MessageDigest md = MessageDigest.getInstance("MD5"); byte[] digest = md.digest(Bytes.toBytes(s)); System.out.println("md5 digest bytes length: " + digest.length); // returns 16 String sDigest = new String(digest); byte[] sbDigest = Bytes.toBytes(sDigest); System.out.println("md5 digest as string length: " + sbDigest.length); // returns 26
Reverse Timestamps A common problem in database processing is quickly finding the most recent version of a value. A technique using reverse timestamps as a part of the key can help greatly with a special case of this problem. Also found in the HBase chapter of Tom White's book Hadoop: The Definitive Guide (O'Reilly), the technique involves appending (Long.MAX_VALUE - timestamp) to the end of any key, e.g., [key][reverse_timestamp]. The most recent value for [key] in a table can be found by performing a Scan for [key] and obtaining the first record. Since HBase keys are in sorted order, this key sorts before any older row-keys for [key] and thus is first. This technique would be used instead of using HBase Versioning where the intent is to hold onto all versions "forever" (or a very long time) and at the same time quickly obtain access to any other version by using the same Scan technique.
Rowkeys and ColumnFamilies Rowkeys are scoped to ColumnFamilies. Thus, the same rowkey could exist in each ColumnFamily that exists in a table without collision.
Immutability of Rowkeys Rowkeys cannot be changed. The only way they can be "changed" in a table is if the row is deleted and then re-inserted. This is a fairly common question on the HBase dist-list so it pays to get the rowkeys right the first time (and/or before you've inserted a lot of data).
Number of Versions
Maximum Number of Versions The maximum number of row versions to store is configured per column family via HColumnDescriptor. The default for max versions is 3. This is an important parameter because as described in section HBase does not overwrite row values, but rather stores different values per row by time (and qualifier). Excess versions are removed during major compactions. The number of max versions may need to be increased or decreased depending on application needs. It is not recommended setting the number of max versions to an exceedingly high level (e.g., hundreds or more) unless those old values are very dear to you because this will greatly increase StoreFile size.
Minimum Number of Versions Like maximum number of row versions, the minimum number of row versions to keep is configured per column family via HColumnDescriptor. The default for min versions is 0, which means the feature is disabled. The minimum number of row versions parameter is used together with the time-to-live parameter and can be combined with the number of row versions parameter to allow configurations such as "keep the last T minutes worth of data, at most N versions, but keep at least M versions around" (where M is the value for minimum number of row versions, M<N). This parameter should only be set when time-to-live is enabled for a column family and must be less than the number of row versions.
Supported Datatypes HBase supports a "bytes-in/bytes-out" interface via Put and Result, so anything that can be converted to an array of bytes can be stored as a value. Input could be strings, numbers, complex objects, or even images as long as they can rendered as bytes. There are practical limits to the size of values (e.g., storing 10-50MB objects in HBase would probably be too much to ask); search the mailling list for conversations on this topic. All rows in HBase conform to the datamodel, and that includes versioning. Take that into consideration when making your design, as well as block size for the ColumnFamily.
Counters One supported datatype that deserves special mention are "counters" (i.e., the ability to do atomic increments of numbers). See Increment in HTable. Synchronization on counters are done on the RegionServer, not in the client.
Joins If you have multiple tables, don't forget to factor in the potential for into the schema design.
Time To Live (TTL) ColumnFamilies can set a TTL length in seconds, and HBase will automatically delete rows once the expiration time is reached. This applies to all versions of a row - even the current one. The TTL time encoded in the HBase for the row is specified in UTC. See HColumnDescriptor for more information.
Keeping Deleted Cells ColumnFamilies can optionally keep deleted cells. That means deleted cells can still be retrieved with Get or Scan operations, as long these operations have a time range specified that ends before the timestamp of any delete that would affect the cells. This allows for point in time queries even in the presence of deletes. Deleted cells are still subject to TTL and there will never be more than "maximum number of versions" deleted cells. A new "raw" scan options returns all deleted rows and the delete markers. See HColumnDescriptor for more information.
Secondary Indexes and Alternate Query Paths This section could also be titled "what if my table rowkey looks like this but I also want to query my table like that." A common example on the dist-list is where a row-key is of the format "user-timestamp" but there are are reporting requirements on activity across users for certain time ranges. Thus, selecting by user is easy because it is in the lead position of the key, but time is not. There is no single answer on the best way to handle this because it depends on... Number of users Data size and data arrival rate Flexibility of reporting requirements (e.g., completely ad-hoc date selection vs. pre-configured ranges) Desired execution speed of query (e.g., 90 seconds may be reasonable to some for an ad-hoc report, whereas it may be too long for others) ... and solutions are also influenced by the size of the cluster and how much processing power you have to throw at the solution. Common techniques are in sub-sections below. This is a comprehensive, but not exhaustive, list of approaches. It should not be a surprise that secondary indexes require additional cluster space and processing. This is precisely what happens in an RDBMS because the act of creating an alternate index requires both space and processing cycles to update. RBDMS products are more advanced in this regard to handle alternative index management out of the box. However, HBase scales better at larger data volumes, so this is a feature trade-off. Pay attention to when implementing any of these approaches. Additionally, see the David Butler response in this dist-list thread HBase, mail # user - Stargate+hbase
Filter Query Depending on the case, it may be appropriate to use . In this case, no secondary index is created. However, don't try a full-scan on a large table like this from an application (i.e., single-threaded client).
Periodic-Update Secondary Index A secondary index could be created in an other table which is periodically updated via a MapReduce job. The job could be executed intra-day, but depending on load-strategy it could still potentially be out of sync with the main data table. See for more information.
Dual-Write Secondary Index Another strategy is to build the secondary index while publishing data to the cluster (e.g., write to data table, write to index table). If this is approach is taken after a data table already exists, then bootstrapping will be needed for the secondary index with a MapReduce job (see ).
Summary Tables Where time-ranges are very wide (e.g., year-long report) and where the data is voluminous, summary tables are a common approach. These would be generated with MapReduce jobs into another table. See for more information.
Coprocessor Secondary Index Coprocessors act like RDBMS triggers. These were added in 0.92. For more information, see
Schema Design Smackdown This section will describe common schema design questions that appear on the dist-list. These are general guidelines and not laws - each application must consider its own needs.
Rows vs. Versions A common question is whether one should prefer rows or HBase's built-in-versioning. The context is typically where there are "a lot" of versions of a row to be retained (e.g., where it is significantly above the HBase default of 3 max versions). The rows-approach would require storing a timstamp in some portion of the rowkey so that they would not overwite with each successive update. Preference: Rows (generally speaking).
Rows vs. Columns Another common question is whether one should prefer rows or columns. The context is typically in extreme cases of wide tables, such as having 1 row with 1 million attributes, or 1 million rows with 1 columns apiece. Preference: Rows (generally speaking). To be clear, this guideline is in the context is in extremely wide cases, not in the standard use-case where one needs to store a few dozen or hundred columns.
Operational and Performance Configuration Options See the Performance section for more information operational and performance schema design options, such as Bloom Filters, Table-configured regionsizes, compression, and blocksizes.
Constraints HBase currently supports 'constraints' in traditional (SQL) database parlance. The advised usage for Constraints is in enforcing business rules for attributes in the table (eg. make sure values are in the range 1-10). Constraints could also be used to enforce referential integrity, but this is strongly discouraged as it will dramatically decrease the write throughput of the tables where integrity checking is enabled. Extensive documentation on using Constraints can be found at: Constraint since version 0.94.
HBase and MapReduce See HBase and MapReduce up in javadocs. Start there. Below is some additional help. For more information about MapReduce (i.e., the framework in general), see the Hadoop MapReduce Tutorial.
Map-Task Spitting
The Default HBase MapReduce Splitter When TableInputFormat is used to source an HBase table in a MapReduce job, its splitter will make a map task for each region of the table. Thus, if there are 100 regions in the table, there will be 100 map-tasks for the job - regardless of how many column families are selected in the Scan.
Custom Splitters For those interested in implementing custom splitters, see the method getSplits in TableInputFormatBase. That is where the logic for map-task assignment resides.
HBase MapReduce Examples
HBase MapReduce Read Example The following is an example of using HBase as a MapReduce source in read-only manner. Specifically, there is a Mapper instance but no Reducer, and nothing is being emitted from the Mapper. There job would be defined as follows... Configuration config = HBaseConfiguration.create(); Job job = new Job(config, "ExampleRead"); job.setJarByClass(MyReadJob.class); // class that contains mapper Scan scan = new Scan(); scan.setCaching(500); // 1 is the default in Scan, which will be bad for MapReduce jobs scan.setCacheBlocks(false); // don't set to true for MR jobs // set other scan attrs ... TableMapReduceUtil.initTableMapperJob( tableName, // input HBase table name scan, // Scan instance to control CF and attribute selection MyMapper.class, // mapper null, // mapper output key null, // mapper output value job); job.setOutputFormatClass(NullOutputFormat.class); // because we aren't emitting anything from mapper boolean b = job.waitForCompletion(true); if (!b) { throw new IOException("error with job!"); } ...and the mapper instance would extend TableMapper... public static class MyMapper extends TableMapper<Text, Text> { public void map(ImmutableBytesWritable row, Result value, Context context) throws InterruptedException, IOException { // process data for the row from the Result instance. } }
HBase MapReduce Read/Write Example The following is an example of using HBase both as a source and as a sink with MapReduce. This example will simply copy data from one table to another. Configuration config = HBaseConfiguration.create(); Job job = new Job(config,"ExampleReadWrite"); job.setJarByClass(MyReadWriteJob.class); // class that contains mapper Scan scan = new Scan(); scan.setCaching(500); // 1 is the default in Scan, which will be bad for MapReduce jobs scan.setCacheBlocks(false); // don't set to true for MR jobs // set other scan attrs TableMapReduceUtil.initTableMapperJob( sourceTable, // input table scan, // Scan instance to control CF and attribute selection MyMapper.class, // mapper class null, // mapper output key null, // mapper output value job); TableMapReduceUtil.initTableReducerJob( targetTable, // output table null, // reducer class job); job.setNumReduceTasks(0); boolean b = job.waitForCompletion(true); if (!b) { throw new IOException("error with job!"); } An explanation is required of what TableMapReduceUtil is doing, especially with the reducer. TableOutputFormat is being used as the outputFormat class, and several parameters are being set on the config (e.g., TableOutputFormat.OUTPUT_TABLE), as well as setting the reducer output key to ImmutableBytesWritable and reducer value to Writable. These could be set by the programmer on the job and conf, but TableMapReduceUtil tries to make things easier. The following is the example mapper, which will create a Put and matching the input Result and emit it. Note: this is what the CopyTable utility does. public static class MyMapper extends TableMapper<ImmutableBytesWritable, Put> { public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException, InterruptedException { // this example is just copying the data from the source table... context.write(row, resultToPut(row,value)); } private static Put resultToPut(ImmutableBytesWritable key, Result result) throws IOException { Put put = new Put(key.get()); for (KeyValue kv : result.raw()) { put.add(kv); } return put; } } There isn't actually a reducer step, so TableOutputFormat takes care of sending the Put to the target table. This is just an example, developers could choose not to use TableOutputFormat and connect to the target table themselves.
HBase MapReduce Read/Write Example With Multi-Table Output TODO: example for MultiTableOutputFormat.
HBase MapReduce Summary to HBase Example The following example uses HBase as a MapReduce source and sink with a summarization step. This example will count the number of distinct instances of a value in a table and write those summarized counts in another table. Configuration config = HBaseConfiguration.create(); Job job = new Job(config,"ExampleSummary"); job.setJarByClass(MySummaryJob.class); // class that contains mapper and reducer Scan scan = new Scan(); scan.setCaching(500); // 1 is the default in Scan, which will be bad for MapReduce jobs scan.setCacheBlocks(false); // don't set to true for MR jobs // set other scan attrs TableMapReduceUtil.initTableMapperJob( sourceTable, // input table scan, // Scan instance to control CF and attribute selection MyMapper.class, // mapper class Text.class, // mapper output key IntWritable.class, // mapper output value job); TableMapReduceUtil.initTableReducerJob( targetTable, // output table MyTableReducer.class, // reducer class job); job.setNumReduceTasks(1); // at least one, adjust as required boolean b = job.waitForCompletion(true); if (!b) { throw new IOException("error with job!"); } In this example mapper a column with a String-value is chosen as the value to summarize upon. This value is used as the key to emit from the mapper, and an IntWritable represents an instance counter. public static class MyMapper extends TableMapper<Text, IntWritable> { private final IntWritable ONE = new IntWritable(1); private Text text = new Text(); public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException, InterruptedException { String val = new String(value.getValue(Bytes.toBytes("cf"), Bytes.toBytes("attr1"))); text.set(val); // we can only emit Writables... context.write(text, ONE); } } In the reducer, the "ones" are counted (just like any other MR example that does this), and then emits a Put. public static class MyTableReducer extends TableReducer<Text, IntWritable, ImmutableBytesWritable> { public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException { int i = 0; for (IntWritable val : values) { i += val.get(); } Put put = new Put(Bytes.toBytes(key.toString())); put.add(Bytes.toBytes("cf"), Bytes.toBytes("count"), Bytes.toBytes(i)); context.write(null, put); } }
HBase MapReduce Summary to File Example This very similar to the summary example above, with exception that this is using HBase as a MapReduce source but HDFS as the sink. The differences are in the job setup and in the reducer. The mapper remains the same. Configuration config = HBaseConfiguration.create(); Job job = new Job(config,"ExampleSummaryToFile"); job.setJarByClass(MySummaryFileJob.class); // class that contains mapper and reducer Scan scan = new Scan(); scan.setCaching(500); // 1 is the default in Scan, which will be bad for MapReduce jobs scan.setCacheBlocks(false); // don't set to true for MR jobs // set other scan attrs TableMapReduceUtil.initTableMapperJob( sourceTable, // input table scan, // Scan instance to control CF and attribute selection MyMapper.class, // mapper class Text.class, // mapper output key IntWritable.class, // mapper output value job); job.setReducerClass(MyReducer.class); // reducer class job.setNumReduceTasks(1); // at least one, adjust as required FileOutputFormat.setOutputPath(job, new Path("/tmp/mr/mySummaryFile")); // adjust directories as required boolean b = job.waitForCompletion(true); if (!b) { throw new IOException("error with job!"); } As stated above, the previous Mapper can run unchanged with this example. As for the Reducer, it is a "generic" Reducer instead of extending TableMapper and emitting Puts. public static class MyReducer extends Reducer<Text, IntWritable, Text, IntWritable> { public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException { int i = 0; for (IntWritable val : values) { i += val.get(); } context.write(key, new IntWritable(i)); } }
HBase MapReduce Summary to HBase Without Reducer It is also possible to perform summaries without a reducer - if you use HBase as the reducer. An HBase target table would need to exist for the job summary. The HTable method incrementColumnValue would be used to atomically increment values. From a performance perspective, it might make sense to keep a Map of values with their values to be incremeneted for each map-task, and make one update per key at during the cleanup method of the mapper. However, your milage may vary depending on the number of rows to be processed and unique keys. In the end, the summary results are in HBase.
HBase MapReduce Summary to RDBMS Sometimes it is more appropriate to generate summaries to an RDBMS. For these cases, it is possible to generate summaries directly to an RDBMS via a custom reducer. The setup method can connect to an RDBMS (the connection information can be passed via custom parameters in the context) and the cleanup method can close the connection. It is critical to understand that number of reducers for the job affects the summarization implementation, and you'll have to design this into your reducer. Specifically, whether it is designed to run as a singleton (one reducer) or multiple reducers. Neither is right or wrong, it depends on your use-case. Recognize that the more reducers that are assigned to the job, the more simultaneous connections to the RDBMS will be created - this will scale, but only to a point. public static class MyRdbmsReducer extends Reducer<Text, IntWritable, Text, IntWritable> { private Connection c = null; public void setup(Context context) { // create DB connection... } public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException { // do summarization // in this example the keys are Text, but this is just an example } public void cleanup(Context context) { // close db connection } } In the end, the summary results are written to your RDBMS table/s.
Accessing Other HBase Tables in a MapReduce Job Although the framework currently allows one HBase table as input to a MapReduce job, other HBase tables can be accessed as lookup tables, etc., in a MapReduce job via creating an HTable instance in the setup method of the Mapper. public class MyMapper extends TableMapper<Text, LongWritable> { private HTable myOtherTable; public void setup(Context context) { myOtherTable = new HTable("myOtherTable"); } public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException, InterruptedException { // process Result... // use 'myOtherTable' for lookups }
Speculative Execution It is generally advisable to turn off speculative execution for MapReduce jobs that use HBase as a source. This can either be done on a per-Job basis through properties, on on the entire cluster. Especially for longer running jobs, speculative execution will create duplicate map-tasks which will double-write your data to HBase; this is probably not what you want. See for more information.
Architecture
Overview
NoSQL? HBase is a type of "NoSQL" database. "NoSQL" is a general term meaning that the database isn't an RDBMS which supports SQL as its primary access language, but there are many types of NoSQL databases: BerkeleyDB is an example of a local NoSQL database, whereas HBase is very much a distributed database. Technically speaking, HBase is really more a "Data Store" than "Data Base" because it lacks many of the features you find in an RDBMS, such as typed columns, secondary indexes, triggers, and advanced query languages, etc. However, HBase has many features which supports both linear and modular scaling. HBase clusters expand by adding RegionServers that are hosted on commodity class servers. If a cluster expands from 10 to 20 RegionServers, for example, it doubles both in terms of storage and as well as processing capacity. RDBMS can scale well, but only up to a point - specifically, the size of a single database server - and for the best performance requires specialized hardware and storage devices. HBase features of note are: Strongly consistent reads/writes: HBase is not an "eventually consistent" DataStore. This makes it very suitable for tasks such as high-speed counter aggregation. Automatic sharding: HBase tables are distributed on the cluster via regions, and regions are automatically split and re-distributed as your data grows. Automatic RegionServer failover Hadoop/HDFS Integration: HBase supports HDFS out of the box as its distributed file system. MapReduce: HBase supports massively parallelized processing via MapReduce for using HBase as both source and sink. Java Client API: HBase supports an easy to use Java API for programmatic access. Thrift/REST API: HBase also supports Thrift and REST for non-Java front-ends. Block Cache and Bloom Filters: HBase supports a Block Cache and Bloom Filters for high volume query optimization. Operational Management: HBase provides build-in web-pages for operational insight as well as JMX metrics.
When Should I Use HBase? HBase isn't suitable for every problem. First, make sure you have enough data. If you have hundreds of millions or billions of rows, then HBase is a good candidate. If you only have a few thousand/million rows, then using a traditional RDBMS might be a better choice due to the fact that all of your data might wind up on a single node (or two) and the rest of the cluster may be sitting idle. Second, make sure you can live without all the extra features that an RDBMS provides (e.g., typed columns, secondary indexes, transactions, advanced query languages, etc.) An application built against an RDBMS cannot be "ported" to HBase by simply changing a JDBC driver, for example. Consider moving from an RDBMS to HBase as a complete redesign as opposed to a port. Third, make sure you have enough hardware. Even HDFS doesn't do well with anything less than 5 DataNodes (due to things such as HDFS block replication which has a default of 3), plus a NameNode. HBase can run quite well stand-alone on a laptop - but this should be considered a development configuration only.
What Is The Difference Between HBase and Hadoop/HDFS? HDFS is a distributed file system that is well suited for the storage of large files. It's documentation states that it is not, however, a general purpose file system, and does not provide fast individual record lookups in files. HBase, on the other hand, is built on top of HDFS and provides fast record lookups (and updates) for large tables. This can sometimes be a point of conceptual confusion. HBase internally puts your data in indexed "StoreFiles" that exist on HDFS for high-speed lookups. See the and the rest of this chapter for more information on how HBase achieves its goals.
Catalog Tables The catalog tables -ROOT- and .META. exist as HBase tables. They are are filtered out of the HBase shell's list command, but they are in fact tables just like any other.
ROOT -ROOT- keeps track of where the .META. table is. The -ROOT- table structure is as follows: Key: .META. region key (.META.,,1) Values: info:regioninfo (serialized HRegionInfo instance of .META.) info:server (server:port of the RegionServer holding .META.) info:serverstartcode (start-time of the RegionServer process holding .META.)
META The .META. table keeps a list of all regions in the system. The .META. table structure is as follows: Key: Region key of the format ([table],[region start key],[region id]) Values: info:regioninfo (serialized HRegionInfo instance for this region) info:server (server:port of the RegionServer containing this region) info:serverstartcode (start-time of the RegionServer process containing this region) When a table is in the process of splitting two other columns will be created, info:splitA and info:splitB which represent the two daughter regions. The values for these columns are also serialized HRegionInfo instances. After the region has been split eventually this row will be deleted. Notes on HRegionInfo: the empty key is used to denote table start and table end. A region with an empty start key is the first region in a table. If region has both an empty start and an empty end key, its the only region in the table In the (hopefully unlikely) event that programmatic processing of catalog metadata is required, see the Writables utility.
Startup Sequencing The META location is set in ROOT first. Then META is updated with server and startcode values. For information on region-RegionServer assignment, see .
Client The HBase client HTable is responsible for finding RegionServers that are serving the particular row range of interest. It does this by querying the .META. and -ROOT- catalog tables (TODO: Explain). After locating the required region(s), the client directly contacts the RegionServer serving that region (i.e., it does not go through the master) and issues the read or write request. This information is cached in the client so that subsequent requests need not go through the lookup process. Should a region be reassigned either by the master load balancer or because a RegionServer has died, the client will requery the catalog tables to determine the new location of the user region. See for more information about the impact of the Master on HBase Client communication. Administrative functions are handled through HBaseAdmin
Connections For connection configuration information, see . HTable instances are not thread-safe. When creating HTable instances, it is advisable to use the same HBaseConfiguration instance. This will ensure sharing of ZooKeeper and socket instances to the RegionServers which is usually what you want. For example, this is preferred: HBaseConfiguration conf = HBaseConfiguration.create(); HTable table1 = new HTable(conf, "myTable"); HTable table2 = new HTable(conf, "myTable"); as opposed to this: HBaseConfiguration conf1 = HBaseConfiguration.create(); HTable table1 = new HTable(conf1, "myTable"); HBaseConfiguration conf2 = HBaseConfiguration.create(); HTable table2 = new HTable(conf2, "myTable"); For more information about how connections are handled in the HBase client, see HConnectionManager.
Connection Pooling For applications which require high-end multithreaded access (e.g., web-servers or application servers that may serve many application threads in a single JVM), see HTablePool.
WriteBuffer and Batch Methods If is turned off on HTable, Puts are sent to RegionServers when the writebuffer is filled. The writebuffer is 2MB by default. Before an HTable instance is discarded, either close() or flushCommits() should be invoked so Puts will not be lost. Note: htable.delete(Delete); does not go in the writebuffer! This only applies to Puts. For additional information on write durability, review the ACID semantics page. For fine-grained control of batching of Puts or Deletes, see the batch methods on HTable.
External Clients Information on non-Java clients and custom protocols is covered in
RowLocks RowLocks are still in the client API however they are discouraged because if not managed properly these can lock up the RegionServers. There is an oustanding ticket HBASE-2332 to remove this feature from the client.
Client Request Filters Get and Scan instances can be optionally configured with filters which are applied on the RegionServer. Filters can be confusing because there are many different types, and it is best to approach them by understanding the groups of Filter functionality.
Structural Structural Filters contain other Filters.
FilterList FilterList represents a list of Filters with a relationship of FilterList.Operator.MUST_PASS_ALL or FilterList.Operator.MUST_PASS_ONE between the Filters. The following example shows an 'or' between two Filters (checking for either 'my value' or 'my other value' on the same attribute). FilterList list = new FilterList(FilterList.Operator.MUST_PASS_ONE); SingleColumnValueFilter filter1 = new SingleColumnValueFilter( cf, column, CompareOp.EQUAL, Bytes.toBytes("my value") ); list.add(filter1); SingleColumnValueFilter filter2 = new SingleColumnValueFilter( cf, column, CompareOp.EQUAL, Bytes.toBytes("my other value") ); list.add(filter2); scan.setFilter(list);
Column Value
SingleColumnValueFilter SingleColumnValueFilter can be used to test column values for equivalence (CompareOp.EQUAL ), inequality (CompareOp.NOT_EQUAL), or ranges (e.g., CompareOp.GREATER). The folowing is example of testing equivalence a column to a String value "my value"... SingleColumnValueFilter filter = new SingleColumnValueFilter( cf, column, CompareOp.EQUAL, Bytes.toBytes("my value") ); scan.setFilter(filter);
Column Value Comparators There are several Comparator classes in the Filter package that deserve special mention. These Comparators are used in concert with other Filters, such as .
RegexStringComparator RegexStringComparator supports regular expressions for value comparisons. RegexStringComparator comp = new RegexStringComparator("my."); // any value that starts with 'my' SingleColumnValueFilter filter = new SingleColumnValueFilter( cf, column, CompareOp.EQUAL, comp ); scan.setFilter(filter); See the Oracle JavaDoc for supported RegEx patterns in Java.
SubstringComparator SubstringComparator can be used to determine if a given substring exists in a value. The comparison is case-insensitive. SubstringComparator comp = new SubstringComparator("y val"); // looking for 'my value' SingleColumnValueFilter filter = new SingleColumnValueFilter( cf, column, CompareOp.EQUAL, comp ); scan.setFilter(filter);
BinaryPrefixComparator See BinaryPrefixComparator.
BinaryComparator See BinaryComparator.
KeyValue Metadata As HBase stores data internally as KeyValue pairs, KeyValue Metadata Filters evaluate the existence of keys (i.e., ColumnFamily:Column qualifiers) for a row, as opposed to values the previous section.
FamilyFilter FamilyFilter can be used to filter on the ColumnFamily. It is generally a better idea to select ColumnFamilies in the Scan than to do it with a Filter.
QualifierFilter QualifierFilter can be used to filter based on Column (aka Qualifier) name.
ColumnPrefixFilter ColumnPrefixFilter can be used to filter based on the lead portion of Column (aka Qualifier) names. A ColumnPrefixFilter seeks ahead to the first column matching the prefix in each row and for each involved column family. It can be used to efficiently get a subset of the columns in very wide rows. Note: The same column qualifier can be used in different column families. This filter returns all matching columns. Example: Find all columns in a row and family that start with "abc" HTableInterface t = ...; byte[] row = ...; byte[] family = ...; byte[] prefix = Bytes.toBytes("abc"); Scan scan = new Scan(row, row); // (optional) limit to one row scan.addFamily(family); // (optional) limit to one family Filter f = new ColumnPrefixFilter(prefix); scan.setFilter(f); scan.setBatch(10); // set this if there could be many columns returned ResultScanner rs = t.getScanner(scan); for (Result r = rs.next(); r != null; r = rs.next()) { for (KeyValue kv : r.raw()) { // each kv represents a column } } rs.close();
MultipleColumnPrefixFilter MultipleColumnPrefixFilter behaves like ColumnPrefixFilter but allows specifying multiple prefixes. Like ColumnPrefixFilter, MultipleColumnPrefixFilter efficiently seeks ahead to the first column matching the lowest prefix and also seeks past ranges of columns between prefixes. It can be used to efficiently get discontinuous sets of columns from very wide rows. Example: Find all columns in a row and family that start with "abc" or "xyz" HTableInterface t = ...; byte[] row = ...; byte[] family = ...; byte[][] prefixes = new byte[][] {Bytes.toBytes("abc"), Bytes.toBytes("xyz")}; Scan scan = new Scan(row, row); // (optional) limit to one row scan.addFamily(family); // (optional) limit to one family Filter f = new MultipleColumnPrefixFilter(prefixes); scan.setFilter(f); scan.setBatch(10); // set this if there could be many columns returned ResultScanner rs = t.getScanner(scan); for (Result r = rs.next(); r != null; r = rs.next()) { for (KeyValue kv : r.raw()) { // each kv represents a column } } rs.close();
ColumnRangeFilter A ColumnRangeFilter allows efficient intra row scanning. A ColumnRangeFilter can seek ahead to the first matching column for each involved column family. It can be used to efficiently get a 'slice' of the columns of a very wide row. i.e. you have a million columns in a row but you only want to look at columns bbbb-bbdd. Note: The same column qualifier can be used in different column families. This filter returns all matching columns. Example: Find all columns in a row and family between "bbbb" (inclusive) and "bbdd" (inclusive) HTableInterface t = ...; byte[] row = ...; byte[] family = ...; byte[] startColumn = Bytes.toBytes("bbbb"); byte[] endColumn = Bytes.toBytes("bbdd"); Scan scan = new Scan(row, row); // (optional) limit to one row scan.addFamily(family); // (optional) limit to one family Filter f = new ColumnRangeFilter(startColumn, true, endColumn, true); scan.setFilter(f); scan.setBatch(10); // set this if there could be many columns returned ResultScanner rs = t.getScanner(scan); for (Result r = rs.next(); r != null; r = rs.next()) { for (KeyValue kv : r.raw()) { // each kv represents a column } } rs.close(); Note: Introduced in HBase 0.92
RowKey
RowFilter It is generally a better idea to use the startRow/stopRow methods on Scan for row selection, however RowFilter can also be used.
Utility
FirstKeyOnlyFilter This is primarily used for rowcount jobs. See FirstKeyOnlyFilter.
Master HMaster is the implementation of the Master Server. The Master server is responsible for monitoring all RegionServer instances in the cluster, and is the interface for all metadata changes. In a distributed cluster, the Master typically runs on the J Mohamed Zahoor goes into some more detail on the Master Architecture in this blog posting, HBase HMaster Architecture .
Startup Behavior If run in a multi-Master environment, all Masters compete to run the cluster. If the active Master loses its lease in ZooKeeper (or the Master shuts down), then then the remaining Masters jostle to take over the Master role.
Runtime Impact A common dist-list question is what happens to an HBase cluster when the Master goes down. Because the HBase client talks directly to the RegionServers, the cluster can still function in a "steady state." Additionally, per ROOT and META exist as HBase tables (i.e., are not resident in the Master). However, the Master controls critical functions such as RegionServer failover and completing region splits. So while the cluster can still run for a time without the Master, the Master should be restarted as soon as possible.
Interface The methods exposed by HMasterInterface are primarily metadata-oriented methods: Table (createTable, modifyTable, removeTable, enable, disable) ColumnFamily (addColumn, modifyColumn, removeColumn) Region (move, assign, unassign) For example, when the HBaseAdmin method disableTable is invoked, it is serviced by the Master server.
Processes The Master runs several background threads:
LoadBalancer Periodically, and when there are not any regions in transition, a load balancer will run and move regions around to balance cluster load. See for configuring this property. See for more information on region assignment.
CatalogJanitor Periodically checks and cleans up the .META. table. See for more information on META.
RegionServer HRegionServer is the RegionServer implementation. It is responsible for serving and managing regions. In a distributed cluster, a RegionServer runs on a .
Interface The methods exposed by HRegionRegionInterface contain both data-oriented and region-maintenance methods: Data (get, put, delete, next, etc.) Region (splitRegion, compactRegion, etc.) For example, when the HBaseAdmin method majorCompact is invoked on a table, the client is actually iterating through all regions for the specified table and requesting a major compaction directly to each region.
Processes The RegionServer runs a variety of background threads:
CompactSplitThread Checks for splits and handle minor compactions.
MajorCompactionChecker Checks for major compactions.
MemStoreFlusher Periodically flushes in-memory writes in the MemStore to StoreFiles.
LogRoller Periodically checks the RegionServer's HLog.
Coprocessors Coprocessors were added in 0.92. There is a thorough Blog Overview of CoProcessors posted. Documentation will eventually move to this reference guide, but the blog is the most current information available at this time.
Block Cache
Design The Block Cache is an LRU cache that contains three levels of block priority to allow for scan-resistance and in-memory ColumnFamilies: Single access priority: The first time a block is loaded from HDFS it normally has this priority and it will be part of the first group to be considered during evictions. The advantage is that scanned blocks are more likely to get evicted than blocks that are getting more usage. Mutli access priority: If a block in the previous priority group is accessed again, it upgrades to this priority. It is thus part of the second group considered during evictions. In-memory access priority: If the block's family was configured to be "in-memory", it will be part of this priority disregarding the number of times it was accessed. Catalog tables are configured like this. This group is the last one considered during evictions. For more information, see the LruBlockCache source
Usage Block caching is enabled by default for all the user tables which means that any read operation will load the LRU cache. This might be good for a large number of use cases, but further tunings are usually required in order to achieve better performance. An important concept is the working set size, or WSS, which is: "the amount of memory needed to compute the answer to a problem". For a website, this would be the data that's needed to answer the queries over a short amount of time. The way to calculate how much memory is available in HBase for caching is: number of region servers * heap size * hfile.block.cache.size * 0.85 The default value for the block cache is 0.25 which represents 25% of the available heap. The last value (85%) is the default acceptable loading factor in the LRU cache after which eviction is started. The reason it is included in this equation is that it would be unrealistic to say that it is possible to use 100% of the available memory since this would make the process blocking from the point where it loads new blocks. Here are some examples: One region server with the default heap size (1GB) and the default block cache size will have 217MB of block cache available. 20 region servers with the heap size set to 8GB and a default block cache size will have 34GB of block cache. 100 region servers with the heap size set to 24GB and a block cache size of 0.5 will have about 1TB of block cache. Your data isn't the only resident of the block cache, here are others that you may have to take into account: Catalog tables: The -ROOT- and .META. tables are forced into the block cache and have the in-memory priority which means that they are harder to evict. The former never uses more than a few hundreds of bytes while the latter can occupy a few MBs (depending on the number of regions). HFiles indexes: HFile is the file format that HBase uses to store data in HDFS and it contains a multi-layered index in order seek to the data without having to read the whole file. The size of those indexes is a factor of the block size (64KB by default), the size of your keys and the amount of data you are storing. For big data sets it's not unusual to see numbers around 1GB per region server, although not all of it will be in cache because the LRU will evict indexes that aren't used. Keys: Taking into account only the values that are being stored is missing half the picture since every value is stored along with its keys (row key, family, qualifier, and timestamp). See . Bloom filters: Just like the HFile indexes, those data structures (when enabled) are stored in the LRU. Currently the recommended way to measure HFile indexes and bloom filters sizes is to look at the region server web UI and checkout the relevant metrics. For keys, sampling can be done by using the HFile command line tool and look for the average key size metric. It's generally bad to use block caching when the WSS doesn't fit in memory. This is the case when you have for example 40GB available across all your region servers' block caches but you need to process 1TB of data. One of the reasons is that the churn generated by the evictions will trigger more garbage collections unnecessarily. Here are two use cases: Fully random reading pattern: This is a case where you almost never access the same row twice within a short amount of time such that the chance of hitting a cached block is close to 0. Setting block caching on such a table is a waste of memory and CPU cycles, more so that it will generate more garbage to pick up by the JVM. For more information on monitoring GC, see . Mapping a table: In a typical MapReduce job that takes a table in input, every row will be read only once so there's no need to put them into the block cache. The Scan object has the option of turning this off via the setCaching method (set it to false). You can still keep block caching turned on on this table if you need fast random read access. An example would be counting the number of rows in a table that serves live traffic, caching every block of that table would create massive churn and would surely evict data that's currently in use.
Write Ahead Log (WAL)
Purpose Each RegionServer adds updates (Puts, Deletes) to its write-ahead log (WAL) first, and then to the for the affected . This ensures that HBase has durable writes. Without WAL, there is the possibility of data loss in the case of a RegionServer failure before each MemStore is flushed and new StoreFiles are written. HLog is the HBase WAL implementation, and there is one HLog instance per RegionServer. The WAL is in HDFS in /hbase/.logs/ with subdirectories per region. For more general information about the concept of write ahead logs, see the Wikipedia Write-Ahead Log article.
WAL Flushing TODO (describe).
WAL Splitting
How edits are recovered from a crashed RegionServer When a RegionServer crashes, it will lose its ephemeral lease in ZooKeeper...TODO
<varname>hbase.hlog.split.skip.errors</varname> When set to true, the default, any error encountered splitting will be logged, the problematic WAL will be moved into the .corrupt directory under the hbase rootdir, and processing will continue. If set to false, the exception will be propagated and the split logged as failed. See HBASE-2958 When hbase.hlog.split.skip.errors is set to false, we fail the split but thats it. We need to do more than just fail split if this flag is set.
How EOFExceptions are treated when splitting a crashed RegionServers' WALs If we get an EOF while splitting logs, we proceed with the split even when hbase.hlog.split.skip.errors == false. An EOF while reading the last log in the set of files to split is near-guaranteed since the RegionServer likely crashed mid-write of a record. But we'll continue even if we got an EOF reading other than the last file in the set. For background, see HBASE-2643 Figure how to deal with eof splitting logs
Regions Regions are the basic element of availability and distribution for tables, and are comprised of a Store per Column Family. The heirarchy of objects is as follows: Table (HBase table) Region (Regions for the table) Store (Store per ColumnFamily for each Region for the table) MemStore (MemStore for each Store for each Region for the table) StoreFile (StoreFiles for each Store for each Region for the table) Block (Blocks within a StoreFile within a Store for each Region for the table) For a description of what HBase files look like when written to HDFS, see .
Region Size Determining the "right" region size can be tricky, and there are a few factors to consider: HBase scales by having regions across many servers. Thus if you have 2 regions for 16GB data, on a 20 node machine your data will be concentrated on just a few machines - nearly the entire cluster will be idle. This really cant be stressed enough, since a common problem is loading 200MB data into HBase then wondering why your awesome 10 node cluster isn't doing anything. On the other hand, high region count has been known to make things slow. This is getting better with each release of HBase, but it is probably better to have 700 regions than 3000 for the same amount of data. There is not much memory footprint difference between 1 region and 10 in terms of indexes, etc, held by the RegionServer. When starting off, its probably best to stick to the default region-size, perhaps going smaller for hot tables (or manually split hot regions to spread the load over the cluster), or go with larger region sizes if your cell sizes tend to be largish (100k and up). See for more information on configuration.
Region-RegionServer Assignment This section describes how Regions are assigned to RegionServers.
Startup When HBase starts regions are assigned as follows (short version): The Master invokes the AssignmentManager upon startup. The AssignmentManager looks at the existing region assignments in META. If the region assignment is still valid (i.e., if the RegionServer is still online) then the assignment is kept. If the assignment is invalid, then the LoadBalancerFactory is invoked to assign the region. The DefaultLoadBalancer will randomly assign the region to a RegionServer. META is updated with the RegionServer assignment (if needed) and the RegionServer start codes (start time of the RegionServer process) upon region opening by the RegionServer.
Failover When a RegionServer fails (short version): The regions immediately become unavailable because the RegionServer is down. The Master will detect that the RegionServer has failed. The region assignments will be considered invalid and will be re-assigned just like the startup sequence.
Region Load Balancing Regions can be periodically moved by the .
Region-RegionServer Locality Over time, Region-RegionServer locality is achieved via HDFS block replication. The HDFS client does the following by default when choosing locations to write replicas: First replica is written to local node Second replica is written to another node in same rack Third replica is written to a node in another rack (if sufficient nodes) Thus, HBase eventually achieves locality for a region after a flush or a compaction. In a RegionServer failover situation a RegionServer may be assigned regions with non-local StoreFiles (because none of the replicas are local), however as new data is written in the region, or the table is compacted and StoreFiles are re-written, they will become "local" to the RegionServer. For more information, see HDFS Design on Replica Placement and also Lars George's blog on HBase and HDFS locality.
Region Splits Splits run unaided on the RegionServer; i.e. the Master does not participate. The RegionServer splits a region, offlines the split region and then adds the daughter regions to META, opens daughters on the parent's hosting RegionServer and then reports the split to the Master. See for how to manually manage splits (and for why you might do this)
Custom Split Policies The default split policy can be overwritten using a custom RegionSplitPolicy (HBase 0.94+). Typically a custom split policy should extend HBase's default split policy: ConstantSizeRegionSplitPolicy. The policy can set globally through the HBaseConfiguration used or on a per table basis: HTableDescriptor myHtd = ...; myHtd.setValue(HTableDescriptor.SPLIT_POLICY, MyCustomSplitPolicy.class.getName());
Store A Store hosts a MemStore and 0 or more StoreFiles (HFiles). A Store corresponds to a column family for a table for a given region.
MemStore The MemStore holds in-memory modifications to the Store. Modifications are KeyValues. When asked to flush, current memstore is moved to snapshot and is cleared. HBase continues to serve edits out of new memstore and backing snapshot until flusher reports in that the flush succeeded. At this point the snapshot is let go.
StoreFile (HFile) StoreFiles are where your data lives.
HFile Format The hfile file format is based on the SSTable file described in the BigTable [2006] paper and on Hadoop's tfile (The unit test suite and the compression harness were taken directly from tfile). Schubert Zhang's blog post on HFile: A Block-Indexed File Format to Store Sorted Key-Value Pairs makes for a thorough introduction to HBase's hfile. Matteo Bertozzi has also put up a helpful description, HBase I/O: HFile. For more information, see the HFile source code. Also see for information about the HFile v2 format that was included in 0.92.
HFile Tool To view a textualized version of hfile content, you can do use the org.apache.hadoop.hbase.io.hfile.HFile tool. Type the following to see usage:$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.io.hfile.HFile For example, to view the content of the file hdfs://10.81.47.41:8020/hbase/TEST/1418428042/DSMP/4759508618286845475, type the following: $ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.io.hfile.HFile -v -f hdfs://10.81.47.41:8020/hbase/TEST/1418428042/DSMP/4759508618286845475 If you leave off the option -v to see just a summary on the hfile. See usage for other things to do with the HFile tool.
StoreFile Directory Structure on HDFS For more information of what StoreFiles look like on HDFS with respect to the directory structure, see .
Blocks StoreFiles are composed of blocks. The blocksize is configured on a per-ColumnFamily basis. Compression happens at the block level within StoreFiles. For more information on compression, see . For more information on blocks, see the HFileBlock source code.
KeyValue The KeyValue class is the heart of data storage in HBase. KeyValue wraps a byte array and takes offsets and lengths into passed array at where to start interpreting the content as KeyValue. The KeyValue format inside a byte array is: keylength valuelength key value The Key is further decomposed as: rowlength row (i.e., the rowkey) columnfamilylength columnfamily columnqualifier timestamp keytype (e.g., Put, Delete, DeleteColumn, DeleteFamily) KeyValue instances are not split across blocks. For example, if there is an 8 MB KeyValue, even if the block-size is 64kb this KeyValue will be read in as a coherent block. For more information, see the KeyValue source code.
Example To emphasize the points above, examine what happens with two Puts for two different columns for the same row: Put #1: rowkey=row1, cf:attr1=value1 Put #2: rowkey=row1, cf:attr2=value2 Even though these are for the same row, a KeyValue is created for each column: Key portion for Put #1: rowlength ------------> 4 row -----------------> row1 columnfamilylength ---> 2 columnfamily --------> cf columnqualifier ------> attr1 timestamp -----------> server time of Put keytype -------------> Put Key portion for Put #2: rowlength ------------> 4 row -----------------> row1 columnfamilylength ---> 2 columnfamily --------> cf columnqualifier ------> attr2 timestamp -----------> server time of Put keytype -------------> Put
It is critical to understand that the rowkey, ColumnFamily, and column (aka columnqualifier) are embedded within the KeyValue instance. The longer these identifiers are, the bigger the KeyValue is.
Compaction There are two types of compactions: minor and major. Minor compactions will usually pick up a couple of the smaller adjacent StoreFiles and rewrite them as one. Minors do not drop deletes or expired cells, only major compactions do this. Sometimes a minor compaction will pick up all the StoreFiles in the Store and in this case it actually promotes itself to being a major compaction. After a major compaction runs there will be a single StoreFile per Store, and this will help performance usually. Caution: major compactions rewrite all of the Stores data and on a loaded system, this may not be tenable; major compactions will usually have to be done manually on large systems. See . Compactions will not perform region merges. See for more information on region merging.
Compaction File Selection To understand the core algorithm for StoreFile selection, there is some ASCII-art in the Store source code that will serve as useful reference. It has been copied below: /* normal skew: * * older ----> newer * _ * | | _ * | | | | _ * --|-|- |-|- |-|---_-------_------- minCompactSize * | | | | | | | | _ | | * | | | | | | | | | | | | * | | | | | | | | | | | | */ Important knobs: hbase.store.compaction.ratio Ratio used in compaction file selection algorithm (default 1.2f). hbase.hstore.compaction.min (.90 hbase.hstore.compactionThreshold) (files) Minimum number of StoreFiles per Store to be selected for a compaction to occur (default 2). hbase.hstore.compaction.max (files) Maximum number of StoreFiles to compact per minor compaction (default 10). hbase.hstore.compaction.min.size (bytes) Any StoreFile smaller than this setting with automatically be a candidate for compaction. Defaults to hbase.hregion.memstore.flush.size (128 mb). hbase.hstore.compaction.max.size (.92) (bytes) Any StoreFile larger than this setting with automatically be excluded from compaction (default Long.MAX_VALUE). The minor compaction StoreFile selection logic is size based, and selects a file for compaction when the file <= sum(smaller_files) * hbase.hstore.compaction.ratio.
Minor Compaction File Selection - Example #1 (Basic Example) This example mirrors an example from the unit test TestCompactSelection. hbase.store.compaction.ratio = 1.0f hbase.hstore.compaction.min = 3 (files) > hbase.hstore.compaction.max = 5 (files) > hbase.hstore.compaction.min.size = 10 (bytes) > hbase.hstore.compaction.max.size = 1000 (bytes) > The following StoreFiles exist: 100, 50, 23, 12, and 12 bytes apiece (oldest to newest). With the above parameters, the files that would be selected for minor compaction are 23, 12, and 12. Why? 100 --> No, because sum(50, 23, 12, 12) * 1.0 = 97. 50 --> No, because sum(23, 12, 12) * 1.0 = 47. 23 --> Yes, because sum(12, 12) * 1.0 = 24. 12 --> Yes, because the previous file has been included, and because this does not exceed the the max-file limit of 5 12 --> Yes, because the previous file had been included, and because this does not exceed the the max-file limit of 5.
Minor Compaction File Selection - Example #2 (Not Enough Files To Compact) This example mirrors an example from the unit test TestCompactSelection. hbase.store.compaction.ratio = 1.0f hbase.hstore.compaction.min = 3 (files) > hbase.hstore.compaction.max = 5 (files) > hbase.hstore.compaction.min.size = 10 (bytes) > hbase.hstore.compaction.max.size = 1000 (bytes) > The following StoreFiles exist: 100, 25, 12, and 12 bytes apiece (oldest to newest). With the above parameters, the files that would be selected for minor compaction are 23, 12, and 12. Why? 100 --> No, because sum(25, 12, 12) * 1.0 = 47 25 --> No, because sum(12, 12) * 1.0 = 24 12 --> No. Candidate because sum(12) * 1.0 = 12, there are only 2 files to compact and that is less than the threshold of 3 12 --> No. Candidate because the previous StoreFile was, but there are not enough files to compact
Minor Compaction File Selection - Example #3 (Limiting Files To Compact) This example mirrors an example from the unit test TestCompactSelection. hbase.store.compaction.ratio = 1.0f hbase.hstore.compaction.min = 3 (files) > hbase.hstore.compaction.max = 5 (files) > hbase.hstore.compaction.min.size = 10 (bytes) > hbase.hstore.compaction.max.size = 1000 (bytes) > The following StoreFiles exist: 7, 6, 5, 4, 3, 2, and 1 bytes apiece (oldest to newest). With the above parameters, the files that would be selected for minor compaction are 7, 6, 5, 4, 3. Why? 7 --> Yes, because sum(6, 5, 4, 3, 2, 1) * 1.0 = 21. Also, 7 is less than the min-size 6 --> Yes, because sum(5, 4, 3, 2, 1) * 1.0 = 15. Also, 6 is less than the min-size. 5 --> Yes, because sum(4, 3, 2, 1) * 1.0 = 10. Also, 5 is less than the min-size. 4 --> Yes, because sum(3, 2, 1) * 1.0 = 6. Also, 4 is less than the min-size. 3 --> Yes, because sum(2, 1) * 1.0 = 3. Also, 3 is less than the min-size. 2 --> No. Candidate because previous file was selected and 2 is less than the min-size, but the max-number of files to compact has been reached. 1 --> No. Candidate because previous file was selected and 1 is less than the min-size, but max-number of files to compact has been reached.
Impact of Key Configuration Options hbase.store.compaction.ratio. A large ratio (e.g., 10) will produce a single giant file. Conversely, a value of .25 will produce behavior similar to the BigTable compaction algorithm - resulting in 4 StoreFiles. hbase.hstore.compaction.min.size. Because this limit represents the "automatic include" limit for all StoreFiles smaller than this value, this value may need to be adjusted downwards in write-heavy environments where many 1 or 2 mb StoreFiles are being flushed, because every file will be targeted for compaction and the resulting files may still be under the min-size and require further compaction, etc.
Bloom Filters Bloom filters were developed over in HBase-1200 Add bloomfilters. For description of the development process -- why static blooms rather than dynamic -- and for an overview of the unique properties that pertain to blooms in HBase, as well as possible future directions, see the Development Process section of the document BloomFilters in HBase attached to HBase-1200. The bloom filters described here are actually version two of blooms in HBase. In versions up to 0.19.x, HBase had a dynamic bloom option based on work done by the European Commission One-Lab Project 034819. The core of the HBase bloom work was later pulled up into Hadoop to implement org.apache.hadoop.io.BloomMapFile. Version 1 of HBase blooms never worked that well. Version 2 is a rewrite from scratch though again it starts with the one-lab work. See also and .
Bloom StoreFile footprint Bloom filters add an entry to the StoreFile general FileInfo data structure and then two extra entries to the StoreFile metadata section.
BloomFilter in the <classname>StoreFile</classname> <classname>FileInfo</classname> data structure FileInfo has a BLOOM_FILTER_TYPE entry which is set to NONE, ROW or ROWCOL.
BloomFilter entries in <classname>StoreFile</classname> metadata BLOOM_FILTER_META holds Bloom Size, Hash Function used, etc. Its small in size and is cached on StoreFile.Reader load BLOOM_FILTER_DATA is the actual bloomfilter data. Obtained on-demand. Stored in the LRU cache, if it is enabled (Its enabled by default).
Bulk Loading
Overview HBase includes several methods of loading data into tables. The most straightforward method is to either use the TableOutputFormat class from a MapReduce job, or use the normal client APIs; however, these are not always the most efficient methods. The bulk load feature uses a MapReduce job to output table data in HBase's internal data format, and then directly loads the generated StoreFiles into a running cluster. Using bulk load will use less CPU and network resources than simply using the HBase API.
Bulk Load Architecture The HBase bulk load process consists of two main steps.
Preparing data via a MapReduce job The first step of a bulk load is to generate HBase data files (StoreFiles) from a MapReduce job using HFileOutputFormat. This output format writes out data in HBase's internal storage format so that they can be later loaded very efficiently into the cluster. In order to function efficiently, HFileOutputFormat must be configured such that each output HFile fits within a single region. In order to do this, jobs whose output will be bulk loaded into HBase use Hadoop's TotalOrderPartitioner class to partition the map output into disjoint ranges of the key space, corresponding to the key ranges of the regions in the table. HFileOutputFormat includes a convenience function, configureIncrementalLoad(), which automatically sets up a TotalOrderPartitioner based on the current region boundaries of a table.
Completing the data load After the data has been prepared using HFileOutputFormat, it is loaded into the cluster using completebulkload. This command line tool iterates through the prepared data files, and for each one determines the region the file belongs to. It then contacts the appropriate Region Server which adopts the HFile, moving it into its storage directory and making the data available to clients. If the region boundaries have changed during the course of bulk load preparation, or between the preparation and completion steps, the completebulkloads utility will automatically split the data files into pieces corresponding to the new boundaries. This process is not optimally efficient, so users should take care to minimize the delay between preparing a bulk load and importing it into the cluster, especially if other clients are simultaneously loading data through other means.
Importing the prepared data using the completebulkload tool After a data import has been prepared, either by using the importtsv tool with the "importtsv.bulk.output" option or by some other MapReduce job using the HFileOutputFormat, the completebulkload tool is used to import the data into the running cluster. The completebulkload tool simply takes the output path where importtsv or your MapReduce job put its results, and the table name to import into. For example: $ hadoop jar hbase-VERSION.jar completebulkload [-c /path/to/hbase/config/hbase-site.xml] /user/todd/myoutput mytable The -c config-file option can be used to specify a file containing the appropriate hbase parameters (e.g., hbase-site.xml) if not supplied already on the CLASSPATH (In addition, the CLASSPATH must contain the directory that has the zookeeper configuration file if zookeeper is NOT managed by HBase). Note: If the target table does not already exist in HBase, this tool will create the table automatically. This tool will run quickly, after which point the new data will be visible in the cluster.
See Also For more information about the referenced utilities, see and .
Advanced Usage Although the importtsv tool is useful in many cases, advanced users may want to generate data programatically, or import data from other formats. To get started doing so, dig into ImportTsv.java and check the JavaDoc for HFileOutputFormat. The import step of the bulk load can also be done programatically. See the LoadIncrementalHFiles class for more information.
HDFS As HBase runs on HDFS (and each StoreFile is written as a file on HDFS), it is important to have an understanding of the HDFS Architecture especially in terms of how it stores files, handles failovers, and replicates blocks. See the Hadoop documentation on HDFS Architecture for more information.
NameNode The NameNode is responsible for maintaining the filesystem metadata. See the above HDFS Architecture link for more information.
DataNode The DataNodes are responsible for storing HDFS blocks. See the above HDFS Architecture link for more information.
FAQ General When should I use HBase? See the in the Architecture chapter. Are there other HBase FAQs? See the FAQ that is up on the wiki, HBase Wiki FAQ. Does HBase support SQL? Not really. SQL-ish support for HBase via Hive is in development, however Hive is based on MapReduce which is not generally suitable for low-latency requests. See the section for examples on the HBase client. How can I find examples of NoSQL/HBase? See the link to the BigTable paper in in the appendix, as well as the other papers. What is the history of HBase? See . Architecture How does HBase handle Region-RegionServer assignment and locality? See . Configuration How can I get started with my first cluster? See . Where can I learn about the rest of the configuration options? See . Schema Design / Data Access How should I design my schema in HBase? See and How can I store (fill in the blank) in HBase? See . How can I handle secondary indexes in HBase? See Can I change a table's rowkeys? This is a very common quesiton. You can't. See . What APIs does HBase support? See , and . MapReduce How can I use MapReduce with HBase? See Performance and Troubleshooting How can I improve HBase cluster performance? See . How can I troubleshoot my HBase cluster? See . Amazon EC2 I am running HBase on Amazon EC2 and... EC2 issues are a special case. See Troubleshooting and Performance sections. Operations How do I manage my HBase cluster? See How do I back up my HBase cluster? See HBase in Action Where can I find interesting videos and presentations on HBase? See hbck In Depth HBaseFsck (hbck) is a tool for checking for region consistency and table integrity problems and repairing a corrupted HBase. It works in two basic modes -- a read-only inconsistency identifying mode and a multi-phase read-write repair mode.
Running hbck to identify inconsistencies To check to see if your HBase cluster has corruptions, run hbck against your HBase cluster: $ ./bin/hbase hbck At the end of the commands output it prints OK or tells you the number of INCONSISTENCIES present. You may also want to run run hbck a few times because some inconsistencies can be transient (e.g. cluster is starting up or a region is splitting). Operationally you may want to run hbck regularly and setup alert (e.g. via nagios) if it repeatedly reports inconsistencies . A run of hbck will report a list of inconsistencies along with a brief description of the regions and tables affected. The using the -details option will report more details including a representative listing of all the splits present in all the tables. $ ./bin/hbase hbck -details
Inconsistencies If after several runs, inconsistencies continue to be reported, you may have encountered a corruption. These should be rare, but in the event they occur newer versions of HBase include the hbck tool enabled with automatic repair options. There are two invariants that when violated create inconsistencies in HBase: HBase’s region consistency invariant is satisfied if every region is assigned and deployed on exactly one region server, and all places where this state kept is in accordance. HBase’s table integrity invariant is satisfied if for each table, every possible row key resolves to exactly one region. Repairs generally work in three phases -- a read-only information gathering phase that identifies inconsistencies, a table integrity repair phase that restores the table integrity invariant, and then finally a region consistency repair phase that restores the region consistency invariant. Starting from version 0.90.0, hbck could detect region consistency problems report on a subset of possible table integrity problems. It also included the ability to automatically fix the most common inconsistency, region assignment and deployment consistency problems. This repair could be done by using the -fix command line option. These problems close regions if they are open on the wrong server or on multiple region servers and also assigns regions to region servers if they are not open. Starting from HBase versions 0.90.7, 0.92.2 and 0.94.0, several new command line options are introduced to aid repairing a corrupted HBase. This hbck sometimes goes by the nickname “uberhbck”. Each particular version of uber hbck is compatible with the HBase’s of the same major version (0.90.7 uberhbck can repair a 0.90.4). However, versions <=0.90.6 and versions <=0.92.1 may require restarting the master or failing over to a backup master.
Localized repairs When repairing a corrupted HBase, it is best to repair the lowest risk inconsistencies first. These are generally region consistency repairs -- localized single region repairs, that only modify in-memory data, ephemeral zookeeper data, or patch holes in the META table. Region consistency requires that the HBase instance has the state of the region’s data in HDFS (.regioninfo files), the region’s row in the .META. table., and region’s deployment/assignments on region servers and the master in accordance. Options for repairing region consistency include: -fixAssignments (equivalent to the 0.90 -fix option) repairs unassigned, incorrectly assigned or multiply assigned regions. -fixMeta which removes meta rows when corresponding regions are not present in HDFS and adds new meta rows if they regions are present in HDFS while not in META. To fix deployment and assignment problems you can run this command: $ ./bin/hbase hbck -fixAssignments To fix deployment and assignment problems as well as repairing incorrect meta rows you can run this command:. $ ./bin/hbase hbck -fixAssignments -fixMeta There are a few classes of table integrity problems that are low risk repairs. The first two are degenerate (startkey == endkey) regions and backwards regions (startkey > endkey). These are automatically handled by sidelining the data to a temporary directory (/hbck/xxxx). The third low-risk class is hdfs region holes. This can be repaired by using the: -fixHdfsHoles option for fabricating new empty regions on the file system. If holes are detected you can use -fixHdfsHoles and should include -fixMeta and -fixAssignments to make the new region consistent. $ ./bin/hbase hbck -fixAssignments -fixMeta -fixHdfsHoles Since this is a common operation, we’ve added a the -repairHoles flag that is equivalent to the previous command: $ ./bin/hbase hbck -repairHoles If inconsistencies still remain after these steps, you most likely have table integrity problems related to orphaned or overlapping regions.
Region Overlap Repairs Table integrity problems can require repairs that deal with overlaps. This is a riskier operation because it requires modifications to the file system, requires some decision making, and may require some manual steps. For these repairs it is best to analyze the output of a hbck -details run so that you isolate repairs attempts only upon problems the checks identify. Because this is riskier, there are safeguard that should be used to limit the scope of the repairs. WARNING: This is a relatively new and have only been tested on online but idle HBase instances (no reads/writes). Use at your own risk in an active production environment! The options for repairing table integrity violations include: -fixHdfsOrphans option for “adopting” a region directory that is missing a region metadata file (the .regioninfo file). -fixHdfsOverlaps ability for fixing overlapping regions When repairing overlapping regions, a region’s data can be modified on the file system in two ways: 1) by merging regions into a larger region or 2) by sidelining regions by moving data to “sideline” directory where data could be restored later. Merging a large number of regions is technically correct but could result in an extremely large region that requires series of costly compactions and splitting operations. In these cases, it is probably better to sideline the regions that overlap with the most other regions (likely the largest ranges) so that merges can happen on a more reasonable scale. Since these sidelined regions are already laid out in HBase’s native directory and HFile format, they can be restored by using HBase’s bulk load mechanism. The default safeguard thresholds are conservative. These options let you override the default thresholds and to enable the large region sidelining feature. -maxMerge <n> maximum number of overlapping regions to merge -sidelineBigOverlaps if more than maxMerge regions are overlapping, sideline attempt to sideline the regions overlapping with the most other regions. -maxOverlapsToSideline <n> if sidelining large overlapping regions, sideline at most n regions. Since often times you would just want to get the tables repaired, you can use this option to turn on all repair options: -repair includes all the region consistency options and only the hole repairing table integrity options. Finally, there are safeguards to limit repairs to only specific tables. For example the following command would only attempt to repair table TableFoo and TableBar. $ ./bin/hbase/ hbck -repair TableFoo TableBar
Special cases: Meta is not properly assigned There are a few special cases that hbck can handle as well. Sometimes the meta table’s only region is inconsistently assigned or deployed. In this case there is a special -fixMetaOnly option that can try to fix meta assignments. $ ./bin/hbase hbck -fixMetaOnly -fixAssignments
Special cases: HBase version file is missing HBase’s data on the file system requires a version file in order to start. If this flie is missing, you can use the -fixVersionFile option to fabricating a new HBase version file. This assumes that the version of hbck you are running is the appropriate version for the HBase cluster.
Special case: Root and META are corrupt. The most drastic corruption scenario is the case where the ROOT or META is corrupted and HBase will not start. In this case you can use the OfflineMetaRepair tool create new ROOT and META regions and tables. This tool assumes that HBase is offline. It then marches through the existing HBase home directory, loads as much information from region metadata files (.regioninfo files) as possible from the file system. If the region metadata has proper table integrity, it sidelines the original root and meta table directories, and builds new ones with pointers to the region directories and their data. $ ./bin/hbase org.apache.hadoop.hbase.util.OfflineMetaRepair NOTE: This tool is not as clever as uberhbck but can be used to bootstrap repairs that uberhbck can complete. If the tool succeeds you should be able to start hbase and run online repairs if necessary.
Compression In HBase<indexterm><primary>Compression</primary></indexterm>
CompressionTest Tool HBase includes a tool to test compression is set up properly. To run it, type /bin/hbase org.apache.hadoop.hbase.util.CompressionTest. This will emit usage on how to run the tool.
<varname> hbase.regionserver.codecs </varname> To have a RegionServer test a set of codecs and fail-to-start if any code is missing or misinstalled, add the configuration hbase.regionserver.codecs to your hbase-site.xml with a value of codecs to test on startup. For example if the hbase.regionserver.codecs value is lzo,gz and if lzo is not present or improperly installed, the misconfigured RegionServer will fail to start. Administrators might make use of this facility to guard against the case where a new server is added to cluster but the cluster requires install of a particular coded.
LZO Unfortunately, HBase cannot ship with LZO because of the licensing issues; HBase is Apache-licensed, LZO is GPL. Therefore LZO install is to be done post-HBase install. See the Using LZO Compression wiki page for how to make LZO work with HBase. A common problem users run into when using LZO is that while initial setup of the cluster runs smooth, a month goes by and some sysadmin goes to add a machine to the cluster only they'll have forgotten to do the LZO fixup on the new machine. In versions since HBase 0.90.0, we should fail in a way that makes it plain what the problem is, but maybe not. See for a feature to help protect against failed LZO install.
GZIP GZIP will generally compress better than LZO though slower. For some setups, better compression may be preferred. Java will use java's GZIP unless the native Hadoop libs are available on the CLASSPATH; in this case it will use native compressors instead (If the native libs are NOT present, you will see lots of Got brand-new compressor reports in your logs; see ).
SNAPPY If snappy is installed, HBase can make use of it (courtesy of hadoop-snappy See Alejandro's note up on the list on difference between Snappy in Hadoop and Snappy in HBase). Build and install snappy on all nodes of your cluster. Use CompressionTest to verify snappy support is enabled and the libs can be loaded ON ALL NODES of your cluster: $ hbase org.apache.hadoop.hbase.util.CompressionTest hdfs://host/path/to/hbase snappy Create a column family with snappy compression and verify it in the hbase shell: $ hbase> create 't1', { NAME => 'cf1', COMPRESSION => 'SNAPPY' } hbase> describe 't1' In the output of the "describe" command, you need to ensure it lists "COMPRESSION => 'SNAPPY'"
Changing Compression Schemes A frequent question on the dist-list is how to change compression schemes for ColumnFamilies. This is actually quite simple, and can be done via an alter command. Because the compression scheme is encoded at the block-level in StoreFiles, the table does not need to be re-created and the data does not copied somewhere else. Just make sure the old codec is still available until you are sure that all of the old StoreFiles have been compacted.
<link xlink:href="https://github.com/brianfrankcooper/YCSB/">YCSB: The Yahoo! Cloud Serving Benchmark</link> and HBase TODO: Describe how YCSB is poor for putting up a decent cluster load. TODO: Describe setup of YCSB for HBase Ted Dunning redid YCSB so its mavenized and added facility for verifying workloads. See Ted Dunning's YCSB. HFile format version 2
Motivation Note: this feature was introduced in HBase 0.92 We found it necessary to revise the HFile format after encountering high memory usage and slow startup times caused by large Bloom filters and block indexes in the region server. Bloom filters can get as large as 100 MB per HFile, which adds up to 2 GB when aggregated over 20 regions. Block indexes can grow as large as 6 GB in aggregate size over the same set of regions. A region is not considered opened until all of its block index data is loaded. Large Bloom filters produce a different performance problem: the first get request that requires a Bloom filter lookup will incur the latency of loading the entire Bloom filter bit array. To speed up region server startup we break Bloom filters and block indexes into multiple blocks and write those blocks out as they fill up, which also reduces the HFile writer’s memory footprint. In the Bloom filter case, “filling up a block” means accumulating enough keys to efficiently utilize a fixed-size bit array, and in the block index case we accumulate an “index block” of the desired size. Bloom filter blocks and index blocks (we call these “inline blocks”) become interspersed with data blocks, and as a side effect we can no longer rely on the difference between block offsets to determine data block length, as it was done in version 1. HFile is a low-level file format by design, and it should not deal with application-specific details such as Bloom filters, which are handled at StoreFile level. Therefore, we call Bloom filter blocks in an HFile "inline" blocks. We also supply HFile with an interface to write those inline blocks. Another format modification aimed at reducing the region server startup time is to use a contiguous “load-on-open” section that has to be loaded in memory at the time an HFile is being opened. Currently, as an HFile opens, there are separate seek operations to read the trailer, data/meta indexes, and file info. To read the Bloom filter, there are two more seek operations for its “data” and “meta” portions. In version 2, we seek once to read the trailer and seek again to read everything else we need to open the file from a contiguous block.
HFile format version 1 overview As we will be discussing the changes we are making to the HFile format, it is useful to give a short overview of the previous (HFile version 1) format. An HFile in the existing format is structured as follows: HFile Version 1 HFile Version 1 Image courtesy of Lars George, hbase-architecture-101-storage.html.
Block index format in version 1 The block index in version 1 is very straightforward. For each entry, it contains: Offset (long) Uncompressed size (int) Key (a serialized byte array written using Bytes.writeByteArray) Key length as a variable-length integer (VInt) Key bytes The number of entries in the block index is stored in the fixed file trailer, and has to be passed in to the method that reads the block index. One of the limitations of the block index in version 1 is that it does not provide the compressed size of a block, which turns out to be necessary for decompression. Therefore, the HFile reader has to infer this compressed size from the offset difference between blocks. We fix this limitation in version 2, where we store on-disk block size instead of uncompressed size, and get uncompressed size from the block header.
HBase file format with inline blocks (version 2)
Overview The version of HBase introducing the above features reads both version 1 and 2 HFiles, but only writes version 2 HFiles. A version 2 HFile is structured as follows: HFile Version 2 HFile Version 2
Unified version 2 block format In the version 2 every block in the data section contains the following fields: 8 bytes: Block type, a sequence of bytes equivalent to version 1's "magic records". Supported block types are: DATA – data blocks LEAF_INDEX – leaf-level index blocks in a multi-level-block-index BLOOM_CHUNK – Bloom filter chunks META – meta blocks (not used for Bloom filters in version 2 anymore) INTERMEDIATE_INDEX – intermediate-level index blocks in a multi-level blockindex ROOT_INDEX – root>level index blocks in a multi>level block index FILE_INFO – the “file info” block, a small key>value map of metadata BLOOM_META – a Bloom filter metadata block in the load>on>open section TRAILER – a fixed>size file trailer. As opposed to the above, this is not an HFile v2 block but a fixed>size (for each HFile version) data structure INDEX_V1 – this block type is only used for legacy HFile v1 block Compressed size of the block's data, not including the header (int). Can be used for skipping the current data block when scanning HFile data. Uncompressed size of the block's data, not including the header (int) This is equal to the compressed size if the compression algorithm is NON File offset of the previous block of the same type (long) Can be used for seeking to the previous data/index block Compressed data (or uncompressed data if the compression algorithm is NONE). The above format of blocks is used in the following HFile sections: Scanned block section. The section is named so because it contains all data blocks that need to be read when an HFile is scanned sequentially.  Also contains leaf block index and Bloom chunk blocks. Non-scanned block section. This section still contains unified-format v2 blocks but it does not have to be read when doing a sequential scan. This section contains “meta” blocks and intermediate-level index blocks. We are supporting “meta” blocks in version 2 the same way they were supported in version 1, even though we do not store Bloom filter data in these blocks anymore.
Block index in version 2 There are three types of block indexes in HFile version 2, stored in two different formats (root and non-root): Data index — version 2 multi-level block index, consisting of: Version 2 root index, stored in the data block index section of the file Optionally, version 2 intermediate levels, stored in the non%root format in the data index section of the file. Intermediate levels can only be present if leaf level blocks are present Optionally, version 2 leaf levels, stored in the non%root format inline with data blocks Meta index — version 2 root index format only, stored in the meta index section of the file Bloom index — version 2 root index format only, stored in the “load-on-open” section as part of Bloom filter metadata.
Root block index format in version 2 This format applies to: Root level of the version 2 data index Entire meta and Bloom indexes in version 2, which are always single-level. A version 2 root index block is a sequence of entries of the following format, similar to entries of a version 1 block index, but storing on-disk size instead of uncompressed size. Offset (long) This offset may point to a data block or to a deeper>level index block. On-disk size (int) Key (a serialized byte array stored using Bytes.writeByteArray) Key (VInt) Key bytes A single-level version 2 block index consists of just a single root index block. To read a root index block of version 2, one needs to know the number of entries. For the data index and the meta index the number of entries is stored in the trailer, and for the Bloom index it is stored in the compound Bloom filter metadata. For a multi-level block index we also store the following fields in the root index block in the load-on-open section of the HFile, in addition to the data structure described above: Middle leaf index block offset Middle leaf block on-disk size (meaning the leaf index block containing the reference to the “middle” data block of the file) The index of the mid-key (defined below) in the middle leaf-level block. These additional fields are used to efficiently retrieve the mid-key of the HFile used in HFile splits, which we define as the first key of the block with a zero-based index of (n – 1) / 2, if the total number of blocks in the HFile is n. This definition is consistent with how the mid-key was determined in HFile version 1, and is reasonable in general, because blocks are likely to be the same size on average, but we don’t have any estimates on individual key/value pair sizes. When writing a version 2 HFile, the total number of data blocks pointed to by every leaf-level index block is kept track of. When we finish writing and the total number of leaf-level blocks is determined, it is clear which leaf-level block contains the mid-key, and the fields listed above are computed.  When reading the HFile and the mid-key is requested, we retrieve the middle leaf index block (potentially from the block cache) and get the mid-key value from the appropriate position inside that leaf block.
Non-root block index format in version 2 This format applies to intermediate-level and leaf index blocks of a version 2 multi-level data block index. Every non-root index block is structured as follows. numEntries: the number of entries (int). entryOffsets: the “secondary index” of offsets of entries in the block, to facilitate a quick binary search on the key (numEntries + 1 int values). The last value is the total length of all entries in this index block. For example, in a non-root index block with entry sizes 60, 80, 50 the “secondary index” will contain the following int array: {0, 60, 140, 190}. Entries. Each entry contains: Offset of the block referenced by this entry in the file (long) On>disk size of the referenced block (int) Key. The length can be calculated from entryOffsets.
Bloom filters in version 2 In contrast with version 1, in a version 2 HFile Bloom filter metadata is stored in the load-on-open section of the HFile for quick startup. A compound Bloom filter. Bloom filter version = 3 (int). There used to be a DynamicByteBloomFilter class that had the Bloom filter version number 2 The total byte size of all compound Bloom filter chunks (long) Number of hash functions (int Type of hash functions (int) The total key count inserted into the Bloom filter (long) The maximum total number of keys in the Bloom filter (long) The number of chunks (int) Comparator class used for Bloom filter keys, a UTF>8 encoded string stored using Bytes.writeByteArray Bloom block index in the version 2 root block index format
File Info format in versions 1 and 2 The file info block is a serialized HbaseMapWritable (essentially a map from byte arrays to byte arrays) with the following keys, among others. StoreFile-level logic adds more keys to this. hfile.LASTKEY The last key of the file (byte array) hfile.AVG_KEY_LEN The average key length in the file (int) hfile.AVG_VALUE_LEN The average value length in the file (int) File info format did not change in version 2. However, we moved the file info to the final section of the file, which can be loaded as one block at the time the HFile is being opened. Also, we do not store comparator in the version 2 file info anymore. Instead, we store it in the fixed file trailer. This is because we need to know the comparator at the time of parsing the load-on-open section of the HFile.
Fixed file trailer format differences between versions 1 and 2 The following table shows common and different fields between fixed file trailers in versions 1 and 2. Note that the size of the trailer is different depending on the version, so it is “fixed” only within one version. However, the version is always stored as the last four-byte integer in the file. Version 1 Version 2 File info offset (long) Data index offset (long) loadOnOpenOffset (long) The offset of the section that we need toload when opening the file. Number of data index entries (int) metaIndexOffset (long) This field is not being used by the version 1 reader, so we removed it from version 2. uncompressedDataIndexSize (long) The total uncompressed size of the whole data block index, including root-level, intermediate-level, and leaf-level blocks. Number of meta index entries (int) Total uncompressed bytes (long) numEntries (int) numEntries (long) Compression codec: 0 = LZO, 1 = GZ, 2 = NONE (int) The number of levels in the data block index (int) firstDataBlockOffset (long) The offset of the first first data block. Used when scanning. lastDataBlockEnd (long) The offset of the first byte after the last key/value data block. We don't need to go beyond this offset when scanning. Version: 1 (int) Version: 2 (int)
Other Information About HBase
HBase Videos Introduction to HBase Introduction to HBase by Todd Lipcon (Chicago Data Summit 2011). Introduction to HBase by Todd Lipcon (2010). Building Real Time Services at Facebook with HBase by Jonathan Gray (Hadoop World 2011). HBase and Hadoop, Mixing Real-Time and Batch Processing at StumbleUpon by JD Cryans (Hadoop World 2010).
HBase Presentations (Slides) Advanced HBase Schema Design by Lars George (Hadoop World 2011). Introduction to HBase by Todd Lipcon (Chicago Data Summit 2011). Getting The Most From Your HBase Install by Ryan Rawson, Jonathan Gray (Hadoop World 2009).
HBase Papers BigTable by Google (2006). HBase and HDFS Locality by Lars George (2010). No Relation: The Mixed Blessings of Non-Relational Databases by Ian Varley (2009).
HBase Sites Cloudera's HBase Blog has a lot of links to useful HBase information. CAP Confusion is a relevant entry for background information on distributed storage systems. HBase Wiki has a page with a number of presentations.
HBase Books HBase: The Definitive Guide by Lars George.
Hadoop Books Hadoop: The Definitive Guide by Tom White.
HBase History 2006: BigTable paper published by Google. 2006 (end of year): HBase development starts. 2008: HBase becomes Hadoop sub-project. 2010: HBase becomes Apache top-level project. HBase and the Apache Software Foundation HBase is a project in the Apache Software Foundation and as such there are responsibilities to the ASF to ensure a healthy project.
ASF Development Process See the Apache Development Process page for all sorts of information on how the ASF is structured (e.g., PMC, committers, contributors), to tips on contributing and getting involved, and how open-source works at ASF.
ASF Board Reporting Once a quarter, each project in the ASF portfolio submits a report to the ASF board. This is done by the HBase project lead and the committers. See ASF board reporting for more information.
Index