3588 lines
193 KiB
XML
3588 lines
193 KiB
XML
<?xml version="1.0" encoding="UTF-8"?>
|
||
<!--
|
||
/**
|
||
* Copyright 2010 The Apache Software Foundation
|
||
*
|
||
* Licensed to the Apache Software Foundation (ASF) under one
|
||
* or more contributor license agreements. See the NOTICE file
|
||
* distributed with this work for additional information
|
||
* regarding copyright ownership. The ASF licenses this file
|
||
* to you under the Apache License, Version 2.0 (the
|
||
* "License"); you may not use this file except in compliance
|
||
* with the License. You may obtain a copy of the License at
|
||
*
|
||
* http://www.apache.org/licenses/LICENSE-2.0
|
||
*
|
||
* Unless required by applicable law or agreed to in writing, software
|
||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||
* See the License for the specific language governing permissions and
|
||
* limitations under the License.
|
||
*/
|
||
-->
|
||
<book version="5.0" xmlns="http://docbook.org/ns/docbook"
|
||
xmlns:xlink="http://www.w3.org/1999/xlink"
|
||
xmlns:xi="http://www.w3.org/2001/XInclude"
|
||
xmlns:svg="http://www.w3.org/2000/svg"
|
||
xmlns:m="http://www.w3.org/1998/Math/MathML"
|
||
xmlns:html="http://www.w3.org/1999/xhtml"
|
||
xmlns:db="http://docbook.org/ns/docbook" xml:id="book">
|
||
<info>
|
||
|
||
<title><link xlink:href="http://www.hbase.org">
|
||
Apache HBase Reference Guide
|
||
</link></title>
|
||
<subtitle><link xlink:href="http://www.hbase.org">
|
||
<inlinemediaobject>
|
||
<imageobject>
|
||
<imagedata align="middle" valign="middle" fileref="hbase_logo.png" />
|
||
</imageobject>
|
||
</inlinemediaobject>
|
||
</link>
|
||
</subtitle>
|
||
<copyright><year>2012</year><holder>Apache Software Foundation</holder></copyright>
|
||
<abstract>
|
||
<para>This is the official reference guide of
|
||
<link xlink:href="http://www.hbase.org">Apache HBase</link>,
|
||
a distributed, versioned, column-oriented database built on top of
|
||
<link xlink:href="http://hadoop.apache.org/">Apache Hadoop</link> and
|
||
<link xlink:href="http://zookeeper.apache.org/">Apache ZooKeeper</link>.
|
||
</para>
|
||
</abstract>
|
||
|
||
<revhistory>
|
||
<revision>
|
||
<revnumber>
|
||
<?eval ${project.version}?>
|
||
</revnumber>
|
||
<date>
|
||
<?eval ${buildDate}?>
|
||
</date>
|
||
</revision>
|
||
</revhistory>
|
||
</info>
|
||
|
||
<!--XInclude some chapters-->
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="preface.xml" />
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="getting_started.xml" />
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="configuration.xml" />
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="upgrading.xml" />
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="shell.xml" />
|
||
|
||
<chapter xml:id="datamodel">
|
||
<title>Data Model</title>
|
||
<para>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.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
|
||
<section xml:id="conceptual.view"><title>Conceptual View</title>
|
||
<para>
|
||
The following example is a slightly modified form of the one on page
|
||
2 of the <link xlink:href="http://research.google.com/archive/bigtable.html">BigTable</link> paper.
|
||
There is a table called <varname>webtable</varname> that contains two column families named
|
||
<varname>contents</varname> and <varname>anchor</varname>.
|
||
In this example, <varname>anchor</varname> contains two
|
||
columns (<varname>anchor:cssnsi.com</varname>, <varname>anchor:my.look.ca</varname>)
|
||
and <varname>contents</varname> contains one column (<varname>contents:html</varname>).
|
||
<note>
|
||
<title>Column Names</title>
|
||
<para>
|
||
By convention, a column name is made of its column family prefix and a
|
||
<emphasis>qualifier</emphasis>. For example, the
|
||
column
|
||
<emphasis>contents:html</emphasis> is of the column family <varname>contents</varname>
|
||
The colon character (<literal
|
||
moreinfo="none">:</literal>) delimits the column family from the
|
||
column family <emphasis>qualifier</emphasis>.
|
||
</para>
|
||
</note>
|
||
<table frame='all'><title>Table <varname>webtable</varname></title>
|
||
<tgroup cols='4' align='left' colsep='1' rowsep='1'>
|
||
<colspec colname='c1'/>
|
||
<colspec colname='c2'/>
|
||
<colspec colname='c3'/>
|
||
<colspec colname='c4'/>
|
||
<thead>
|
||
<row><entry>Row Key</entry><entry>Time Stamp</entry><entry>ColumnFamily <varname>contents</varname></entry><entry>ColumnFamily <varname>anchor</varname></entry></row>
|
||
</thead>
|
||
<tbody>
|
||
<row><entry>"com.cnn.www"</entry><entry>t9</entry><entry></entry><entry><varname>anchor:cnnsi.com</varname> = "CNN"</entry></row>
|
||
<row><entry>"com.cnn.www"</entry><entry>t8</entry><entry></entry><entry><varname>anchor:my.look.ca</varname> = "CNN.com"</entry></row>
|
||
<row><entry>"com.cnn.www"</entry><entry>t6</entry><entry><varname>contents:html</varname> = "<html>..."</entry><entry></entry></row>
|
||
<row><entry>"com.cnn.www"</entry><entry>t5</entry><entry><varname>contents:html</varname> = "<html>..."</entry><entry></entry></row>
|
||
<row><entry>"com.cnn.www"</entry><entry>t3</entry><entry><varname>contents:html</varname> = "<html>..."</entry><entry></entry></row>
|
||
</tbody>
|
||
</tgroup>
|
||
</table>
|
||
</para>
|
||
</section>
|
||
<section xml:id="physical.view"><title>Physical View</title>
|
||
<para>
|
||
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., <varname>columnfamily:column</varname>) can be added to any
|
||
column family without pre-announcing them.
|
||
<table frame='all'><title>ColumnFamily <varname>anchor</varname></title>
|
||
<tgroup cols='3' align='left' colsep='1' rowsep='1'>
|
||
<colspec colname='c1'/>
|
||
<colspec colname='c2'/>
|
||
<colspec colname='c3'/>
|
||
<thead>
|
||
<row><entry>Row Key</entry><entry>Time Stamp</entry><entry>Column Family <varname>anchor</varname></entry></row>
|
||
</thead>
|
||
<tbody>
|
||
<row><entry>"com.cnn.www"</entry><entry>t9</entry><entry><varname>anchor:cnnsi.com</varname> = "CNN"</entry></row>
|
||
<row><entry>"com.cnn.www"</entry><entry>t8</entry><entry><varname>anchor:my.look.ca</varname> = "CNN.com"</entry></row>
|
||
</tbody>
|
||
</tgroup>
|
||
</table>
|
||
<table frame='all'><title>ColumnFamily <varname>contents</varname></title>
|
||
<tgroup cols='3' align='left' colsep='1' rowsep='1'>
|
||
<colspec colname='c1'/>
|
||
<colspec colname='c2'/>
|
||
<colspec colname='c3'/>
|
||
<thead>
|
||
<row><entry>Row Key</entry><entry>Time Stamp</entry><entry>ColumnFamily "contents:"</entry></row>
|
||
</thead>
|
||
<tbody>
|
||
<row><entry>"com.cnn.www"</entry><entry>t6</entry><entry><varname>contents:html</varname> = "<html>..."</entry></row>
|
||
<row><entry>"com.cnn.www"</entry><entry>t5</entry><entry><varname>contents:html</varname> = "<html>..."</entry></row>
|
||
<row><entry>"com.cnn.www"</entry><entry>t3</entry><entry><varname>contents:html</varname> = "<html>..."</entry></row>
|
||
</tbody>
|
||
</tgroup>
|
||
</table>
|
||
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 <varname>contents:html</varname>
|
||
column at time stamp <literal>t8</literal> would return no value. Similarly, a
|
||
request for an <varname>anchor:my.look.ca</varname> value at time stamp
|
||
<literal>t9</literal> 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
|
||
<varname>com.cnn.www</varname> if no timestamp is specified would be:
|
||
the value of <varname>contents:html</varname> from time stamp
|
||
<literal>t6</literal>, the value of <varname>anchor:cnnsi.com</varname>
|
||
from time stamp <literal>t9</literal>, the value of
|
||
<varname>anchor:my.look.ca</varname> from time stamp <literal>t8</literal>.
|
||
</para>
|
||
<para>For more information about the internals of how HBase stores data, see <xref linkend="regions.arch" />.
|
||
</para>
|
||
</section>
|
||
|
||
<section xml:id="table">
|
||
<title>Table</title>
|
||
<para>
|
||
Tables are declared up front at schema definition time.
|
||
</para>
|
||
</section>
|
||
|
||
<section xml:id="row">
|
||
<title>Row</title>
|
||
<para>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.</para>
|
||
</section>
|
||
|
||
<section xml:id="columnfamily">
|
||
<title>Column Family<indexterm><primary>Column Family</primary></indexterm></title>
|
||
<para>
|
||
Columns in HBase are grouped into <emphasis>column families</emphasis>.
|
||
All column members of a column family have the same prefix. For example, the
|
||
columns <emphasis>courses:history</emphasis> and
|
||
<emphasis>courses:math</emphasis> are both members of the
|
||
<emphasis>courses</emphasis> column family.
|
||
The colon character (<literal
|
||
moreinfo="none">:</literal>) delimits the column family from the
|
||
<indexterm>column family <emphasis>qualifier</emphasis><primary>Column Family Qualifier</primary></indexterm>.
|
||
The column family prefix must be composed of
|
||
<emphasis>printable</emphasis> characters. The qualifying tail, the
|
||
column family <emphasis>qualifier</emphasis>, 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.</para>
|
||
<para>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.</para>
|
||
|
||
<para></para>
|
||
</section>
|
||
<section xml:id="cells">
|
||
<title>Cells<indexterm><primary>Cells</primary></indexterm></title>
|
||
<para>A <emphasis>{row, column, version} </emphasis>tuple exactly
|
||
specifies a <literal>cell</literal> in HBase.
|
||
Cell content is uninterrpreted bytes</para>
|
||
</section>
|
||
<section xml:id="data_model_operations">
|
||
<title>Data Model Operations</title>
|
||
<para>The four primary data model operations are Get, Put, Scan, and Delete. Operations are applied via
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html">HTable</link> instances.
|
||
</para>
|
||
<section xml:id="get">
|
||
<title>Get</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html">Get</link> returns
|
||
attributes for a specified row. Gets are executed via
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#get%28org.apache.hadoop.hbase.client.Get%29">
|
||
HTable.get</link>.
|
||
</para>
|
||
</section>
|
||
<section xml:id="put">
|
||
<title>Put</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Put.html">Put</link> 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
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#put%28org.apache.hadoop.hbase.client.Put%29">
|
||
HTable.put</link> (writeBuffer) or <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#batch%28java.util.List%29">
|
||
HTable.batch</link> (non-writeBuffer).
|
||
</para>
|
||
</section>
|
||
<section xml:id="scan">
|
||
<title>Scans</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">Scan</link> allow
|
||
iteration over multiple rows for specified attributes.
|
||
</para>
|
||
<para>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".
|
||
<programlisting>
|
||
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!
|
||
}
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
<section xml:id="delete">
|
||
<title>Delete</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Delete.html">Delete</link> removes
|
||
a row from a table. Deletes are executed via
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#delete%28org.apache.hadoop.hbase.client.Delete%29">
|
||
HTable.delete</link>.
|
||
</para>
|
||
<para>HBase does not modify data in place, and so deletes are handled by creating new markers called <emphasis>tombstones</emphasis>.
|
||
These tombstones, along with the dead values, are cleaned up on major compactions.
|
||
</para>
|
||
<para>See <xref linkend="version.delete"/> for more information on deleting versions of columns, and see
|
||
<xref linkend="compaction"/> for more information on compactions.
|
||
</para>
|
||
|
||
</section>
|
||
|
||
</section>
|
||
|
||
|
||
<section xml:id="versions">
|
||
<title>Versions<indexterm><primary>Versions</primary></indexterm></title>
|
||
|
||
<para>A <emphasis>{row, column, version} </emphasis>tuple exactly
|
||
specifies a <literal>cell</literal> 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.</para>
|
||
|
||
<para>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
|
||
<code>java.util.Date.getTime()</code> or
|
||
<code>System.currentTimeMillis()</code>, that is: <quote>the difference,
|
||
measured in milliseconds, between the current time and midnight, January
|
||
1, 1970 UTC</quote>.</para>
|
||
|
||
<para>The HBase version dimension is stored in decreasing order, so that
|
||
when reading from a store file, the most recent values are found
|
||
first.</para>
|
||
|
||
<para>There is a lot of confusion over the semantics of
|
||
<literal>cell</literal> versions, in HBase. In particular, a couple
|
||
questions that often come up are:<itemizedlist>
|
||
<listitem>
|
||
<para>If multiple writes to a cell have the same version, are all
|
||
versions maintained or just the last?<footnote>
|
||
<para>Currently, only the last written is fetchable.</para>
|
||
</footnote></para>
|
||
</listitem>
|
||
|
||
<listitem>
|
||
<para>Is it OK to write cells in a non-increasing version
|
||
order?<footnote>
|
||
<para>Yes</para>
|
||
</footnote></para>
|
||
</listitem>
|
||
</itemizedlist></para>
|
||
|
||
<para>Below we describe how the version dimension in HBase currently
|
||
works<footnote>
|
||
<para>See <link
|
||
xlink:href="https://issues.apache.org/jira/browse/HBASE-2406">HBASE-2406</link>
|
||
for discussion of HBase versions. <link
|
||
xlink:href="http://outerthought.org/blog/417-ot.html">Bending time
|
||
in HBase</link> 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
|
||
<emphasis>Overwriting values at existing timestamps</emphasis>
|
||
mentioned in the article no longer holds in HBase. This section is
|
||
basically a synopsis of this article by Bruno Dumon.</para>
|
||
</footnote>.</para>
|
||
|
||
<section xml:id="versions.ops">
|
||
<title>Versions and HBase Operations</title>
|
||
|
||
<para>In this section we look at the behavior of the version dimension
|
||
for each of the core HBase operations.</para>
|
||
|
||
<section>
|
||
<title>Get/Scan</title>
|
||
|
||
<para>Gets are implemented on top of Scans. The below discussion of
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html">Get</link> applies equally to <link
|
||
xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">Scans</link>.</para>
|
||
|
||
<para>By default, i.e. if you specify no explicit version, when
|
||
doing a <literal>get</literal>, 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:</para>
|
||
|
||
<itemizedlist>
|
||
<listitem>
|
||
<para>to return more than one version, see <link
|
||
xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html#setMaxVersions()">Get.setMaxVersions()</link></para>
|
||
</listitem>
|
||
|
||
<listitem>
|
||
<para>to return versions other than the latest, see <link
|
||
xlink:href="???">Get.setTimeRange()</link></para>
|
||
|
||
<para>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.</para>
|
||
</listitem>
|
||
</itemizedlist>
|
||
|
||
</section>
|
||
<section xml:id="default_get_example">
|
||
<title>Default Get Example</title>
|
||
<para>The following Get will only retrieve the current version of the row
|
||
<programlisting>
|
||
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
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
<section xml:id="versioned_get_example">
|
||
<title>Versioned Get Example</title>
|
||
<para>The following Get will return the last 3 versions of the row.
|
||
<programlisting>
|
||
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
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
|
||
<section>
|
||
<title>Put</title>
|
||
|
||
<para>Doing a put always creates a new version of a
|
||
<literal>cell</literal>, at a certain timestamp. By default the
|
||
system uses the server's <literal>currentTimeMillis</literal>, 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.</para>
|
||
|
||
<para>To overwrite an existing value, do a put at exactly the same
|
||
row, column, and version as that of the cell you would
|
||
overshadow.</para>
|
||
<section xml:id="implicit_version_example">
|
||
<title>Implicit Version Example</title>
|
||
<para>The following Put will be implicitly versioned by HBase with the current time.
|
||
<programlisting>
|
||
Put put = new Put(Bytes.toBytes(row));
|
||
put.add(Bytes.toBytes("cf"), Bytes.toBytes("attr1"), Bytes.toBytes( data));
|
||
htable.put(put);
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
<section xml:id="explicit_version_example">
|
||
<title>Explicit Version Example</title>
|
||
<para>The following Put has the version timestamp explicitly set.
|
||
<programlisting>
|
||
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);
|
||
</programlisting>
|
||
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.
|
||
</para>
|
||
</section>
|
||
|
||
</section>
|
||
|
||
<section xml:id="version.delete">
|
||
<title>Delete</title>
|
||
|
||
<para>There are three different types of internal delete markers
|
||
<footnote><para>See Lars Hofhansl's blog for discussion of his attempt
|
||
adding another, <link xlink:href="http://hadoop-hbase.blogspot.com/2012/01/scanning-in-hbase.html">Scanning in HBase: Prefix Delete Marker</link></para></footnote>:
|
||
<itemizedlist>
|
||
<listitem><para>Delete: for a specific version of a column.</para>
|
||
</listitem>
|
||
<listitem><para>Delete column: for all versions of a column.</para>
|
||
</listitem>
|
||
<listitem><para>Delete family: for all columns of a particular ColumnFamily</para>
|
||
</listitem>
|
||
</itemizedlist>
|
||
When deleting an entire row, HBase will internally create a tombstone for each ColumnFamily (i.e., not each individual column).
|
||
</para>
|
||
<para>Deletes work by creating <emphasis>tombstone</emphasis>
|
||
markers. For example, let's suppose we want to delete a row. For
|
||
this you can specify a version, or else by default the
|
||
<literal>currentTimeMillis</literal> is used. What this means is
|
||
<quote>delete all cells where the version is less than or equal to
|
||
this version</quote>. 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 <emphasis>tombstone</emphasis> is
|
||
written, which will mask the deleted values<footnote>
|
||
<para>When HBase does a major compaction, the tombstones are
|
||
processed to actually remove the dead values, together with the
|
||
tombstones themselves.</para>
|
||
</footnote>. 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.</para>
|
||
<para>For an informative discussion on how deletes and versioning interact, see
|
||
the thread <link xlink:href="http://comments.gmane.org/gmane.comp.java.hadoop.hbase.user/28421">Put w/ timestamp -> Deleteall -> Put w/ timestamp fails</link>
|
||
up on the user mailing list.</para>
|
||
<para>Also see <xref linkend="keyvalue"/> for more information on the internal KeyValue format.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
|
||
<section>
|
||
<title>Current Limitations</title>
|
||
|
||
<para>There are still some bugs (or at least 'undecided behavior')
|
||
with the version dimension that will be addressed by later HBase
|
||
releases.</para>
|
||
|
||
<section>
|
||
<title>Deletes mask Puts</title>
|
||
|
||
<para>Deletes mask puts, even puts that happened after the delete
|
||
was entered<footnote>
|
||
<para><link
|
||
xlink:href="https://issues.apache.org/jira/browse/HBASE-2256">HBASE-2256</link></para>
|
||
</footnote>. 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.</para>
|
||
</section>
|
||
|
||
<section>
|
||
<title>Major compactions change query results</title>
|
||
|
||
<para><quote>...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...<footnote>
|
||
<para>See <emphasis>Garbage Collection</emphasis> in <link
|
||
xlink:href="http://outerthought.org/blog/417-ot.html">Bending
|
||
time in HBase</link> </para>
|
||
</footnote></quote></para>
|
||
</section>
|
||
</section>
|
||
</section>
|
||
<section xml:id="dm.sort">
|
||
<title>Sort Order</title>
|
||
<para>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).
|
||
</para>
|
||
</section>
|
||
<section xml:id="dm.column.metadata">
|
||
<title>Column Metadata</title>
|
||
<para>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.
|
||
</para>
|
||
<para>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 <xref linkend="keyvalue" />.
|
||
</para>
|
||
</section>
|
||
<section xml:id="joins"><title>Joins</title>
|
||
<para>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.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
</section>
|
||
</chapter> <!-- data model -->
|
||
|
||
<chapter xml:id="schema">
|
||
<title>HBase and Schema Design</title>
|
||
<para>A good general introduction on the strength and weaknesses modelling on
|
||
the various non-rdbms datastores is Ian Varleys' Master thesis,
|
||
<link xlink:href="http://ianvarley.com/UT/MR/Varley_MastersReport_Full_2009-08-07.pdf">No Relation: The Mixed Blessings of Non-Relational Databases</link>.
|
||
Recommended. Also, read <xref linkend="keyvalue"/> for how HBase stores data internally.
|
||
</para>
|
||
<section xml:id="schema.creation">
|
||
<title>
|
||
Schema Creation
|
||
</title>
|
||
<para>HBase schemas can be created or updated with <xref linkend="shell" />
|
||
or by using <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html">HBaseAdmin</link> in the Java API.
|
||
</para>
|
||
<para>Tables must be disabled when making ColumnFamily modifications, for example..
|
||
<programlisting>
|
||
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);
|
||
</programlisting>
|
||
</para>See <xref linkend="client_dependencies"/> for more information about configuring client connections.
|
||
<para>Note: online schema changes are supported in the 0.92.x codebase, but the 0.90.x codebase requires the table
|
||
to be disabled.
|
||
</para>
|
||
<section xml:id="schema.updates"><title>Schema Updates</title>
|
||
<para>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.
|
||
</para>
|
||
<para>See <xref linkend="store"/> for more information on StoreFiles.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="number.of.cfs">
|
||
<title>
|
||
On the number of column families
|
||
</title>
|
||
<para>
|
||
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 <xref linkend="compaction"/>.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
<section xml:id="number.of.cfs.card"><title>Cardinality of ColumnFamilies</title>
|
||
<para>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.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="rowkey.design"><title>Rowkey Design</title>
|
||
<section xml:id="timeseries">
|
||
<title>
|
||
Monotonically Increasing Row Keys/Timeseries Data
|
||
</title>
|
||
<para>
|
||
In the HBase chapter of Tom White's book <link xlink:url="http://oreilly.com/catalog/9780596521981">Hadoop: The Definitive Guide</link> (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:
|
||
<link xlink:href="http://ikaisays.com/2011/01/25/app-engine-datastore-tip-monotonically-increasing-values-are-bad/">monotonically increasing values are bad</link>. 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.
|
||
</para>
|
||
|
||
|
||
<para>If you do need to upload time series data into HBase, you should
|
||
study <link xlink:href="http://opentsdb.net/">OpenTSDB</link> as a
|
||
successful example. It has a page describing the <link xlink:href=" http://opentsdb.net/schema.html">schema</link> 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 <emphasis>lead</emphasis> 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.
|
||
</para>
|
||
</section>
|
||
<section xml:id="keysize">
|
||
<title>Try to minimize row and column sizes</title>
|
||
<subtitle>Or why are my StoreFile indices large?</subtitle>
|
||
<para>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
|
||
<link xlink:url="https://issues.apache.org/jira/browse/HBASE-3551?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13005272#comment-13005272">HBASE-3551</link>
|
||
(recommended!).
|
||
Therein, the indices that are kept on HBase storefiles (<xref linkend="hfile" />)
|
||
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 <link xlink:href="http://search-hadoop.com/m/hemBv1LiN4Q1/a+question+storefileIndexSize&subj=a+question+storefileIndexSize">a question storefileIndexSize</link>
|
||
up on the user mailing list.
|
||
</para>
|
||
<para>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. </para>
|
||
<para>See <xref linkend="keyvalue"/> for more information on HBase stores data internally to see why this is important.</para>
|
||
<section xml:id="keysize.cf"><title>Column Families</title>
|
||
<para>Try to keep the ColumnFamily names as small as possible, preferably one character (e.g. "d" for data/default).
|
||
</para>
|
||
<para>See <xref linkend="keyvalue"/> for more information on HBase stores data internally to see why this is important.</para>
|
||
</section>
|
||
<section xml:id="keysize.atttributes"><title>Attributes</title>
|
||
<para>Although verbose attribute names (e.g., "myVeryImportantAttribute") are easier to read, prefer shorter attribute names (e.g., "via")
|
||
to store in HBase.
|
||
</para>
|
||
<para>See <xref linkend="keyvalue"/> for more information on HBase stores data internally to see why this is important.</para>
|
||
</section>
|
||
<section xml:id="keysize.row"><title>Rowkey Length</title>
|
||
<para>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.
|
||
</para>
|
||
</section>
|
||
<section xml:id="keysize.patterns"><title>Byte Patterns</title>
|
||
<para>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.
|
||
</para>
|
||
<para>Not convinced? Below is some sample code that you can run on your own.
|
||
<programlisting>
|
||
// 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
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
|
||
</section>
|
||
<section xml:id="reverse.timestamp"><title>Reverse Timestamps</title>
|
||
<para>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 (<code>Long.MAX_VALUE - timestamp</code>) to the end of any key, e.g., [key][reverse_timestamp].
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
<para>This technique would be used instead of using <xref linkend="schema.versions">HBase Versioning</xref> 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.
|
||
</para>
|
||
</section>
|
||
<section xml:id="rowkey.scope">
|
||
<title>Rowkeys and ColumnFamilies</title>
|
||
<para>Rowkeys are scoped to ColumnFamilies. Thus, the same rowkey could exist in each ColumnFamily that exists in a table without collision.
|
||
</para>
|
||
</section>
|
||
<section xml:id="changing.rowkeys"><title>Immutability of Rowkeys</title>
|
||
<para>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).
|
||
</para>
|
||
</section>
|
||
</section> <!-- rowkey design -->
|
||
<section xml:id="schema.versions">
|
||
<title>
|
||
Number of Versions
|
||
</title>
|
||
<section xml:id="schema.versions.max"><title>Maximum Number of Versions</title>
|
||
<para>The maximum number of row versions to store is configured per column
|
||
family via <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</link>.
|
||
The default for max versions is 3.
|
||
This is an important parameter because as described in <xref linkend="datamodel" />
|
||
section HBase does <emphasis>not</emphasis> 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.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
</section>
|
||
<section xml:id="schema.minversions">
|
||
<title>
|
||
Minimum Number of Versions
|
||
</title>
|
||
<para>Like maximum number of row versions, the minimum number of row versions to keep is configured per column
|
||
family via <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</link>.
|
||
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, <emphasis>but keep at least M versions around</emphasis>"
|
||
(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.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="supported.datatypes">
|
||
<title>
|
||
Supported Datatypes
|
||
</title>
|
||
<para>HBase supports a "bytes-in/bytes-out" interface via <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Put.html">Put</link> and
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Result.html">Result</link>, 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.
|
||
</para>
|
||
<para>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 <xref linkend="datamodel">datamodel</xref>, and
|
||
that includes versioning. Take that into consideration when making your design, as well as block size for the ColumnFamily.
|
||
</para>
|
||
<section xml:id="counters">
|
||
<title>Counters</title>
|
||
<para>
|
||
One supported datatype that deserves special mention are "counters" (i.e., the ability to do atomic increments of numbers). See
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#increment%28org.apache.hadoop.hbase.client.Increment%29">Increment</link> in HTable.
|
||
</para>
|
||
<para>Synchronization on counters are done on the RegionServer, not in the client.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="schema.joins"><title>Joins</title>
|
||
<para>If you have multiple tables, don't forget to factor in the potential for <xref linkend="joins"/> into the schema design.
|
||
</para>
|
||
</section>
|
||
<section xml:id="ttl">
|
||
<title>Time To Live (TTL)</title>
|
||
<para>ColumnFamilies can set a TTL length in seconds, and HBase will automatically delete rows once the expiration time is reached.
|
||
This applies to <emphasis>all</emphasis> versions of a row - even the current one. The TTL time encoded in the HBase for the row is specified in UTC.
|
||
</para>
|
||
<para>See <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</link> for more information.
|
||
</para>
|
||
</section>
|
||
<section xml:id="cf.keep.deleted">
|
||
<title>
|
||
Keeping Deleted Cells
|
||
</title>
|
||
<para>ColumnFamilies can optionally keep deleted cells. That means deleted cells can still be retrieved with
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html">Get</link> or
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">Scan</link> 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.
|
||
</para>
|
||
<para>
|
||
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.
|
||
</para>
|
||
<para>See <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html">HColumnDescriptor</link> for more information.
|
||
</para>
|
||
</section>
|
||
<section xml:id="secondary.indexes">
|
||
<title>
|
||
Secondary Indexes and Alternate Query Paths
|
||
</title>
|
||
<para>This section could also be titled "what if my table rowkey looks like <emphasis>this</emphasis> but I also want to query my table like <emphasis>that</emphasis>."
|
||
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.
|
||
</para>
|
||
<para>There is no single answer on the best way to handle this because it depends on...
|
||
<itemizedlist>
|
||
<listitem>Number of users</listitem>
|
||
<listitem>Data size and data arrival rate</listitem>
|
||
<listitem>Flexibility of reporting requirements (e.g., completely ad-hoc date selection vs. pre-configured ranges) </listitem>
|
||
<listitem>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) </listitem>
|
||
</itemizedlist>
|
||
... 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.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
<para>Pay attention to <xref linkend="performance"/> when implementing any of these approaches.</para>
|
||
<para>Additionally, see the David Butler response in this dist-list thread <link xlink:href="http://search-hadoop.com/m/nvbiBp2TDP/Stargate%252Bhbase&subj=Stargate+hbase">HBase, mail # user - Stargate+hbase</link>
|
||
</para>
|
||
<section xml:id="secondary.indexes.filter">
|
||
<title>
|
||
Filter Query
|
||
</title>
|
||
<para>Depending on the case, it may be appropriate to use <xref linkend="client.filter"/>. 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).
|
||
</para>
|
||
</section>
|
||
<section xml:id="secondary.indexes.periodic">
|
||
<title>
|
||
Periodic-Update Secondary Index
|
||
</title>
|
||
<para>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.</para>
|
||
<para>See <xref linkend="mapreduce.example.readwrite"/> for more information.</para>
|
||
</section>
|
||
<section xml:id="secondary.indexes.dualwrite">
|
||
<title>
|
||
Dual-Write Secondary Index
|
||
</title>
|
||
<para>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 <xref linkend="secondary.indexes.periodic"/>).</para>
|
||
</section>
|
||
<section xml:id="secondary.indexes.summary">
|
||
<title>
|
||
Summary Tables
|
||
</title>
|
||
<para>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.</para>
|
||
<para>See <xref linkend="mapreduce.example.summary"/> for more information.</para>
|
||
</section>
|
||
<section xml:id="secondary.indexes.coproc">
|
||
<title>
|
||
Coprocessor Secondary Index
|
||
</title>
|
||
<para>Coprocessors act like RDBMS triggers. These were added in 0.92. For more information, see <xref linkend="coprocessors"/>
|
||
</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="schema.smackdown"><title>Schema Design Smackdown</title>
|
||
<para>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.
|
||
</para>
|
||
<section xml:id="schema.smackdown.rowsversions"><title>Rows vs. Versions</title>
|
||
<para>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.
|
||
</para>
|
||
<para>Preference: Rows (generally speaking).
|
||
</para>
|
||
</section>
|
||
<section xml:id="schema.smackdown.rowscols"><title>Rows vs. Columns</title>
|
||
<para>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.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="schema.ops"><title>Operational and Performance Configuration Options</title>
|
||
<para>See the Performance section <xref linkend="perf.schema"/> for more information operational and performance
|
||
schema design options, such as Bloom Filters, Table-configured regionsizes, compression, and blocksizes.
|
||
</para>
|
||
</section>
|
||
|
||
<section xml:id="constraints"><title>Constraints</title>
|
||
<para>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: <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/constraint">Constraint</link> since version 0.94.
|
||
</para>
|
||
</section>
|
||
|
||
</chapter> <!-- schema design -->
|
||
|
||
<chapter xml:id="mapreduce">
|
||
<title>HBase and MapReduce</title>
|
||
<para>See <link xlink:href="http://hbase.org/apidocs/org/apache/hadoop/hbase/mapreduce/package-summary.html#package_description">
|
||
HBase and MapReduce</link> up in javadocs.
|
||
Start there. Below is some additional help.</para>
|
||
<para>For more information about MapReduce (i.e., the framework in general), see the
|
||
<link xlink:href="http://hadoop.apache.org/common/docs/current/mapred_tutorial.html">Hadoop MapReduce Tutorial</link>.</para>
|
||
<section xml:id="splitter">
|
||
<title>Map-Task Spitting</title>
|
||
<section xml:id="splitter.default">
|
||
<title>The Default HBase MapReduce Splitter</title>
|
||
<para>When <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html">TableInputFormat</link>
|
||
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.</para>
|
||
</section>
|
||
<section xml:id="splitter.custom">
|
||
<title>Custom Splitters</title>
|
||
<para>For those interested in implementing custom splitters, see the method <code>getSplits</code> in
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.html">TableInputFormatBase</link>.
|
||
That is where the logic for map-task assignment resides.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="mapreduce.example">
|
||
<title>HBase MapReduce Examples</title>
|
||
<section xml:id="mapreduce.example.read">
|
||
<title>HBase MapReduce Read Example</title>
|
||
<para>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...
|
||
<programlisting>
|
||
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!");
|
||
}
|
||
</programlisting>
|
||
...and the mapper instance would extend <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableMapper.html">TableMapper</link>...
|
||
<programlisting>
|
||
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.
|
||
}
|
||
}
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
<section xml:id="mapreduce.example.readwrite">
|
||
<title>HBase MapReduce Read/Write Example</title>
|
||
<para>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.
|
||
<programlisting>
|
||
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!");
|
||
}
|
||
</programlisting>
|
||
An explanation is required of what <classname>TableMapReduceUtil</classname> is doing, especially with the reducer.
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.html">TableOutputFormat</link> 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 <classname>ImmutableBytesWritable</classname> and reducer value to <classname>Writable</classname>.
|
||
These could be set by the programmer on the job and conf, but <classname>TableMapReduceUtil</classname> tries to make things easier.
|
||
<para>The following is the example mapper, which will create a <classname>Put</classname> and matching the input <classname>Result</classname>
|
||
and emit it. Note: this is what the CopyTable utility does.
|
||
</para>
|
||
<programlisting>
|
||
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;
|
||
}
|
||
}
|
||
</programlisting>
|
||
<para>There isn't actually a reducer step, so <classname>TableOutputFormat</classname> takes care of sending the <classname>Put</classname>
|
||
to the target table.
|
||
</para>
|
||
<para>This is just an example, developers could choose not to use <classname>TableOutputFormat</classname> and connect to the
|
||
target table themselves.
|
||
</para>
|
||
</para>
|
||
</section>
|
||
<section xml:id="mapreduce.example.readwrite.multi">
|
||
<title>HBase MapReduce Read/Write Example With Multi-Table Output</title>
|
||
<para>TODO: example for <classname>MultiTableOutputFormat</classname>.
|
||
</para>
|
||
</section>
|
||
<section xml:id="mapreduce.example.summary">
|
||
<title>HBase MapReduce Summary to HBase Example</title>
|
||
<para>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.
|
||
<programlisting>
|
||
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!");
|
||
}
|
||
</programlisting>
|
||
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 <classname>IntWritable</classname> represents an instance counter.
|
||
<programlisting>
|
||
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);
|
||
}
|
||
}
|
||
</programlisting>
|
||
In the reducer, the "ones" are counted (just like any other MR example that does this), and then emits a <classname>Put</classname>.
|
||
<programlisting>
|
||
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);
|
||
}
|
||
}
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
<section xml:id="mapreduce.example.summary.file">
|
||
<title>HBase MapReduce Summary to File Example</title>
|
||
<para>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.
|
||
</para>
|
||
<programlisting>
|
||
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!");
|
||
}
|
||
</programlisting>
|
||
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.
|
||
<programlisting>
|
||
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));
|
||
}
|
||
}
|
||
</programlisting>
|
||
</section>
|
||
<section xml:id="mapreduce.example.summary.noreducer">
|
||
<title>HBase MapReduce Summary to HBase Without Reducer</title>
|
||
<para>It is also possible to perform summaries without a reducer - if you use HBase as the reducer.
|
||
</para>
|
||
<para>An HBase target table would need to exist for the job summary. The HTable method <code>incrementColumnValue</code>
|
||
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 <code>
|
||
cleanup</code> method of the mapper. However, your milage may vary depending on the number of rows to be processed and
|
||
unique keys.
|
||
</para>
|
||
<para>In the end, the summary results are in HBase.
|
||
</para>
|
||
</section>
|
||
<section xml:id="mapreduce.example.summary.rdbms">
|
||
<title>HBase MapReduce Summary to RDBMS</title>
|
||
<para>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 <code>setup</code> 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.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
<programlisting>
|
||
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
|
||
}
|
||
|
||
}
|
||
</programlisting>
|
||
<para>In the end, the summary results are written to your RDBMS table/s.
|
||
</para>
|
||
</section>
|
||
|
||
</section> <!-- mr examples -->
|
||
<section xml:id="mapreduce.htable.access">
|
||
<title>Accessing Other HBase Tables in a MapReduce Job</title>
|
||
<para>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.
|
||
<programlisting>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
|
||
}
|
||
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
<section xml:id="mapreduce.specex">
|
||
<title>Speculative Execution</title>
|
||
<para>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.
|
||
</para>
|
||
<para>See <xref linkend="spec.ex"/> for more information.
|
||
</para>
|
||
</section>
|
||
</chapter> <!-- mapreduce -->
|
||
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="security.xml" />
|
||
|
||
<chapter xml:id="architecture">
|
||
<title>Architecture</title>
|
||
<section xml:id="arch.overview">
|
||
<title>Overview</title>
|
||
<section xml:id="arch.overview.nosql">
|
||
<title>NoSQL?</title>
|
||
<para>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.
|
||
</para>
|
||
<para>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:
|
||
<itemizedlist>
|
||
<listitem>Strongly consistent reads/writes: HBase is not an "eventually consistent" DataStore. This
|
||
makes it very suitable for tasks such as high-speed counter aggregation. </listitem>
|
||
<listitem>Automatic sharding: HBase tables are distributed on the cluster via regions, and regions are
|
||
automatically split and re-distributed as your data grows.</listitem>
|
||
<listitem>Automatic RegionServer failover</listitem>
|
||
<listitem>Hadoop/HDFS Integration: HBase supports HDFS out of the box as its distributed file system.</listitem>
|
||
<listitem>MapReduce: HBase supports massively parallelized processing via MapReduce for using HBase as both
|
||
source and sink.</listitem>
|
||
<listitem>Java Client API: HBase supports an easy to use Java API for programmatic access.</listitem>
|
||
<listitem>Thrift/REST API: HBase also supports Thrift and REST for non-Java front-ends.</listitem>
|
||
<listitem>Block Cache and Bloom Filters: HBase supports a Block Cache and Bloom Filters for high volume query optimization.</listitem>
|
||
<listitem>Operational Management: HBase provides build-in web-pages for operational insight as well as JMX metrics.</listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
</section>
|
||
|
||
<section xml:id="arch.overview.when">
|
||
<title>When Should I Use HBase?</title>
|
||
<para>HBase isn't suitable for every problem.</para>
|
||
<para>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.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
<para>HBase can run quite well stand-alone on a laptop - but this should be considered a development
|
||
configuration only.
|
||
</para>
|
||
</section>
|
||
<section xml:id="arch.overview.hbasehdfs">
|
||
<title>What Is The Difference Between HBase and Hadoop/HDFS?</title>
|
||
<para><link xlink:href="http://hadoop.apache.org/hdfs/">HDFS</link> 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 <xref linkend="datamodel" /> and the rest of this chapter for more information on how HBase achieves its goals.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
|
||
<section xml:id="arch.catalog">
|
||
<title>Catalog Tables</title>
|
||
<para>The catalog tables -ROOT- and .META. exist as HBase tables. They are are filtered out
|
||
of the HBase shell's <code>list</code> command, but they are in fact tables just like any other.
|
||
</para>
|
||
<section xml:id="arch.catalog.root">
|
||
<title>ROOT</title>
|
||
<para>-ROOT- keeps track of where the .META. table is. The -ROOT- table structure is as follows:
|
||
</para>
|
||
<para>Key:
|
||
<itemizedlist>
|
||
<listitem>.META. region key (<code>.META.,,1</code>)</listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
<para>Values:
|
||
<itemizedlist>
|
||
<listitem><code>info:regioninfo</code> (serialized <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HRegionInfo.html">HRegionInfo</link>
|
||
instance of .META.)</listitem>
|
||
<listitem><code>info:server</code> (server:port of the RegionServer holding .META.)</listitem>
|
||
<listitem><code>info:serverstartcode</code> (start-time of the RegionServer process holding .META.)</listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
</section>
|
||
<section xml:id="arch.catalog.meta">
|
||
<title>META</title>
|
||
<para>The .META. table keeps a list of all regions in the system. The .META. table structure is as follows:
|
||
</para>
|
||
<para>Key:
|
||
<itemizedlist>
|
||
<listitem>Region key of the format (<code>[table],[region start key],[region id]</code>)</listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
<para>Values:
|
||
<itemizedlist>
|
||
<listitem><code>info:regioninfo</code> (serialized <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HRegionInfo.html">
|
||
HRegionInfo</link> instance for this region)
|
||
</listitem>
|
||
<listitem><code>info:server</code> (server:port of the RegionServer containing this region)</listitem>
|
||
<listitem><code>info:serverstartcode</code> (start-time of the RegionServer process containing this region)</listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
<para>When a table is in the process of splitting two other columns will be created, <code>info:splitA</code> and <code>info:splitB</code>
|
||
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.
|
||
</para>
|
||
<para>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
|
||
</para>
|
||
<para>In the (hopefully unlikely) event that programmatic processing of catalog metadata is required, see the
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/util/Writables.html#getHRegionInfo%28byte[]%29">Writables</link> utility.
|
||
</para>
|
||
</section>
|
||
<section xml:id="arch.catalog.startup">
|
||
<title>Startup Sequencing</title>
|
||
<para>The META location is set in ROOT first. Then META is updated with server and startcode values.
|
||
</para>
|
||
<para>For information on region-RegionServer assignment, see <xref linkend="regions.arch.assignment"/>.
|
||
</para>
|
||
</section>
|
||
</section> <!-- catalog -->
|
||
|
||
<section xml:id="client">
|
||
<title>Client</title>
|
||
<para>The HBase client
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html">HTable</link>
|
||
is responsible for finding RegionServers that are serving the
|
||
particular row range of interest. It does this by querying
|
||
the <code>.META.</code> and <code>-ROOT-</code> catalog tables
|
||
(TODO: Explain). After locating the required
|
||
region(s), the client <emphasis>directly</emphasis> 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.
|
||
</para>
|
||
<para>See <xref linkend="master.runtime"/> for more information about the impact of the Master on HBase Client
|
||
communication.
|
||
</para>
|
||
<para>Administrative functions are handled through <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html">HBaseAdmin</link>
|
||
</para>
|
||
<section xml:id="client.connections"><title>Connections</title>
|
||
<para>For connection configuration information, see <xref linkend="client_dependencies" />.
|
||
</para>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html">HTable</link>
|
||
instances are not thread-safe. When creating HTable instances, it is advisable to use the same <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HBaseConfiguration">HBaseConfiguration</link>
|
||
instance. This will ensure sharing of ZooKeeper and socket instances to the RegionServers
|
||
which is usually what you want. For example, this is preferred:
|
||
<programlisting>HBaseConfiguration conf = HBaseConfiguration.create();
|
||
HTable table1 = new HTable(conf, "myTable");
|
||
HTable table2 = new HTable(conf, "myTable");</programlisting>
|
||
as opposed to this:
|
||
<programlisting>HBaseConfiguration conf1 = HBaseConfiguration.create();
|
||
HTable table1 = new HTable(conf1, "myTable");
|
||
HBaseConfiguration conf2 = HBaseConfiguration.create();
|
||
HTable table2 = new HTable(conf2, "myTable");</programlisting>
|
||
For more information about how connections are handled in the HBase client,
|
||
see <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HConnectionManager.html">HConnectionManager</link>.
|
||
</para>
|
||
<section xml:id="client.connection.pooling"><title>Connection Pooling</title>
|
||
<para>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 <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTablePool.html">HTablePool</link>.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="client.writebuffer"><title>WriteBuffer and Batch Methods</title>
|
||
<para>If <xref linkend="perf.hbase.client.autoflush" /> is turned off on
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html">HTable</link>,
|
||
<classname>Put</classname>s are sent to RegionServers when the writebuffer
|
||
is filled. The writebuffer is 2MB by default. Before an HTable instance is
|
||
discarded, either <methodname>close()</methodname> or
|
||
<methodname>flushCommits()</methodname> should be invoked so Puts
|
||
will not be lost.
|
||
</para>
|
||
<para>Note: <code>htable.delete(Delete);</code> does not go in the writebuffer! This only applies to Puts.
|
||
</para>
|
||
<para>For additional information on write durability, review the <link xlink:href="acid-semantics.html">ACID semantics</link> page.
|
||
</para>
|
||
<para>For fine-grained control of batching of
|
||
<classname>Put</classname>s or <classname>Delete</classname>s,
|
||
see the <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#batch%28java.util.List%29">batch</link> methods on HTable.
|
||
</para>
|
||
</section>
|
||
<section xml:id="client.external"><title>External Clients</title>
|
||
<para>Information on non-Java clients and custom protocols is covered in <xref linkend="external_apis" />
|
||
</para>
|
||
</section>
|
||
<section xml:id="client.rowlocks"><title>RowLocks</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html#lockRow%28byte[]%29">RowLocks</link> are still
|
||
in the client API <emphasis>however</emphasis> they are discouraged because if not managed properly these can
|
||
lock up the RegionServers.
|
||
</para>
|
||
<para>There is an oustanding ticket <link xlink:href="https://issues.apache.org/jira/browse/HBASE-2332">HBASE-2332</link> to
|
||
remove this feature from the client.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
|
||
<section xml:id="client.filter"><title>Client Request Filters</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Get.html">Get</link> and <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html">Scan</link> instances can be
|
||
optionally configured with <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/Filter.html">filters</link> which are applied on the RegionServer.
|
||
</para>
|
||
<para>Filters can be confusing because there are many different types, and it is best to approach them by understanding the groups
|
||
of Filter functionality.
|
||
</para>
|
||
<section xml:id="client.filter.structural"><title>Structural</title>
|
||
<para>Structural Filters contain other Filters.</para>
|
||
<section xml:id="client.filter.structural.fl"><title>FilterList</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FilterList.html">FilterList</link>
|
||
represents a list of Filters with a relationship of <code>FilterList.Operator.MUST_PASS_ALL</code> or
|
||
<code>FilterList.Operator.MUST_PASS_ONE</code> 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).
|
||
<programlisting>
|
||
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);
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="client.filter.cv"><title>Column Value</title>
|
||
<section xml:id="client.filter.cv.scvf"><title>SingleColumnValueFilter</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.html">SingleColumnValueFilter</link>
|
||
can be used to test column values for equivalence (<code><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/CompareFilter.CompareOp.html">CompareOp.EQUAL</link>
|
||
</code>), inequality (<code>CompareOp.NOT_EQUAL</code>), or ranges
|
||
(e.g., <code>CompareOp.GREATER</code>). The folowing is example of testing equivalence a column to a String value "my value"...
|
||
<programlisting>
|
||
SingleColumnValueFilter filter = new SingleColumnValueFilter(
|
||
cf,
|
||
column,
|
||
CompareOp.EQUAL,
|
||
Bytes.toBytes("my value")
|
||
);
|
||
scan.setFilter(filter);
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="client.filter.cvp"><title>Column Value Comparators</title>
|
||
<para>There are several Comparator classes in the Filter package that deserve special mention.
|
||
These Comparators are used in concert with other Filters, such as <xref linkend="client.filter.cv.scvf" />.
|
||
</para>
|
||
<section xml:id="client.filter.cvp.rcs"><title>RegexStringComparator</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/RegexStringComparator.html">RegexStringComparator</link>
|
||
supports regular expressions for value comparisons.
|
||
<programlisting>
|
||
RegexStringComparator comp = new RegexStringComparator("my."); // any value that starts with 'my'
|
||
SingleColumnValueFilter filter = new SingleColumnValueFilter(
|
||
cf,
|
||
column,
|
||
CompareOp.EQUAL,
|
||
comp
|
||
);
|
||
scan.setFilter(filter);
|
||
</programlisting>
|
||
See the Oracle JavaDoc for <link xlink:href="http://download.oracle.com/javase/6/docs/api/java/util/regex/Pattern.html">supported RegEx patterns in Java</link>.
|
||
</para>
|
||
</section>
|
||
<section xml:id="client.filter.cvp.rcs"><title>SubstringComparator</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/SubstringComparator.html">SubstringComparator</link>
|
||
can be used to determine if a given substring exists in a value. The comparison is case-insensitive.
|
||
</para>
|
||
<programlisting>
|
||
SubstringComparator comp = new SubstringComparator("y val"); // looking for 'my value'
|
||
SingleColumnValueFilter filter = new SingleColumnValueFilter(
|
||
cf,
|
||
column,
|
||
CompareOp.EQUAL,
|
||
comp
|
||
);
|
||
scan.setFilter(filter);
|
||
</programlisting>
|
||
</section>
|
||
<section xml:id="client.filter.cvp.bfp"><title>BinaryPrefixComparator</title>
|
||
<para>See <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.html">BinaryPrefixComparator</link>.</para>
|
||
</section>
|
||
<section xml:id="client.filter.cvp.bc"><title>BinaryComparator</title>
|
||
<para>See <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/BinaryComparator.html">BinaryComparator</link>.</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="client.filter.kvm"><title>KeyValue Metadata</title>
|
||
<para>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.
|
||
</para>
|
||
<section xml:id="client.filter.kvm.ff"><title>FamilyFilter</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FamilyFilter.html">FamilyFilter</link> 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.</para>
|
||
</section>
|
||
<section xml:id="client.filter.kvm.qf"><title>QualifierFilter</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/QualifierFilter.html">QualifierFilter</link> can be used
|
||
to filter based on Column (aka Qualifier) name.
|
||
</para>
|
||
</section>
|
||
<section xml:id="client.filter.kvm.cpf"><title>ColumnPrefixFilter</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.html">ColumnPrefixFilter</link> can be used
|
||
to filter based on the lead portion of Column (aka Qualifier) names.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
<para>Note: The same column qualifier can be used in different column families. This filter returns all matching columns.
|
||
</para>
|
||
<para>Example: Find all columns in a row and family that start with "abc"
|
||
<programlisting>
|
||
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();
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
<section xml:id="client.filter.kvm.mcpf"><title>MultipleColumnPrefixFilter</title>
|
||
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.html">MultipleColumnPrefixFilter</link> behaves like ColumnPrefixFilter
|
||
but allows specifying multiple prefixes.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
<para>Example: Find all columns in a row and family that start with "abc" or "xyz"
|
||
<programlisting>
|
||
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();
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
<section xml:id="client.filter.kvm.crf "><title>ColumnRangeFilter</title>
|
||
<para>A <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/ColumnRangeFilter.html">ColumnRangeFilter</link> allows efficient intra row scanning.
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
<para>Note: The same column qualifier can be used in different column families. This filter returns all matching columns.
|
||
</para>
|
||
<para>Example: Find all columns in a row and family between "bbbb" (inclusive) and "bbdd" (inclusive)
|
||
<programlisting>
|
||
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();
|
||
</programlisting>
|
||
</para>
|
||
<para>Note: Introduced in HBase 0.92</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="client.filter.row"><title>RowKey</title>
|
||
<section xml:id="client.filter.row.rf"><title>RowFilter</title>
|
||
<para>It is generally a better idea to use the startRow/stopRow methods on Scan for row selection, however
|
||
<link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/RowFilter.html">RowFilter</link> can also be used.</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="client.filter.utility"><title>Utility</title>
|
||
<section xml:id="client.filter.utility.fkof"><title>FirstKeyOnlyFilter</title>
|
||
<para>This is primarily used for rowcount jobs.
|
||
See <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.html">FirstKeyOnlyFilter</link>.</para>
|
||
</section>
|
||
</section>
|
||
</section> <!-- client.filter -->
|
||
|
||
<section xml:id="master"><title>Master</title>
|
||
<para><code>HMaster</code> 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 <xref linkend="arch.hdfs.nn" /><footnote>
|
||
<para>J Mohamed Zahoor goes into some more detail on the Master Architecture in this blog posting, <link
|
||
xlink:href="http://blog.zahoor.in/2012/08/hbase-hmaster-architecture/">HBase HMaster Architecture
|
||
</link>.</para>
|
||
</footnote>
|
||
</para>
|
||
<section xml:id="master.startup"><title>Startup Behavior</title>
|
||
<para>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.
|
||
</para>
|
||
</section>
|
||
<section xml:id="master.runtime"><title>Runtime Impact</title>
|
||
<para>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 <xref linkend="arch.catalog"/> 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 <emphasis>for a time</emphasis> without the Master,
|
||
the Master should be restarted as soon as possible.
|
||
</para>
|
||
</section>
|
||
<section xml:id="master.api"><title>Interface</title>
|
||
<para>The methods exposed by <code>HMasterInterface</code> are primarily metadata-oriented methods:
|
||
<itemizedlist>
|
||
<listitem>Table (createTable, modifyTable, removeTable, enable, disable)
|
||
</listitem>
|
||
<listitem>ColumnFamily (addColumn, modifyColumn, removeColumn)
|
||
</listitem>
|
||
<listitem>Region (move, assign, unassign)
|
||
</listitem>
|
||
</itemizedlist>
|
||
For example, when the <code>HBaseAdmin</code> method <code>disableTable</code> is invoked, it is serviced by the Master server.
|
||
</para>
|
||
</section>
|
||
<section xml:id="master.processes"><title>Processes</title>
|
||
<para>The Master runs several background threads:
|
||
</para>
|
||
<section xml:id="master.processes.loadbalancer"><title>LoadBalancer</title>
|
||
<para>Periodically, and when there are not any regions in transition,
|
||
a load balancer will run and move regions around to balance cluster load.
|
||
See <xref linkend="balancer_config" /> for configuring this property.</para>
|
||
<para>See <xref linkend="regions.arch.assignment"/> for more information on region assignment.
|
||
</para>
|
||
</section>
|
||
<section xml:id="master.processes.catalog"><title>CatalogJanitor</title>
|
||
<para>Periodically checks and cleans up the .META. table. See <xref linkend="arch.catalog.meta" /> for more information on META.</para>
|
||
</section>
|
||
</section>
|
||
|
||
</section>
|
||
<section xml:id="regionserver.arch"><title>RegionServer</title>
|
||
<para><code>HRegionServer</code> is the RegionServer implementation. It is responsible for serving and managing regions.
|
||
In a distributed cluster, a RegionServer runs on a <xref linkend="arch.hdfs.dn" />.
|
||
</para>
|
||
<section xml:id="regionserver.arch.api"><title>Interface</title>
|
||
<para>The methods exposed by <code>HRegionRegionInterface</code> contain both data-oriented and region-maintenance methods:
|
||
<itemizedlist>
|
||
<listitem>Data (get, put, delete, next, etc.)
|
||
</listitem>
|
||
<listitem>Region (splitRegion, compactRegion, etc.)
|
||
</listitem>
|
||
</itemizedlist>
|
||
For example, when the <code>HBaseAdmin</code> method <code>majorCompact</code> 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.
|
||
</para>
|
||
</section>
|
||
<section xml:id="regionserver.arch.processes"><title>Processes</title>
|
||
<para>The RegionServer runs a variety of background threads:</para>
|
||
<section xml:id="regionserver.arch.processes.compactsplit"><title>CompactSplitThread</title>
|
||
<para>Checks for splits and handle minor compactions.</para>
|
||
</section>
|
||
<section xml:id="regionserver.arch.processes.majorcompact"><title>MajorCompactionChecker</title>
|
||
<para>Checks for major compactions.</para>
|
||
</section>
|
||
<section xml:id="regionserver.arch.processes.memstore"><title>MemStoreFlusher</title>
|
||
<para>Periodically flushes in-memory writes in the MemStore to StoreFiles.</para>
|
||
</section>
|
||
<section xml:id="regionserver.arch.processes.log"><title>LogRoller</title>
|
||
<para>Periodically checks the RegionServer's HLog.</para>
|
||
</section>
|
||
</section>
|
||
|
||
<section xml:id="coprocessors"><title>Coprocessors</title>
|
||
<para>Coprocessors were added in 0.92. There is a thorough <link xlink:href="https://blogs.apache.org/hbase/entry/coprocessor_introduction">Blog Overview of CoProcessors</link>
|
||
posted. Documentation will eventually move to this reference guide, but the blog is the most current information available at this time.
|
||
</para>
|
||
</section>
|
||
|
||
<section xml:id="block.cache">
|
||
<title>Block Cache</title>
|
||
<section xml:id="block.cache.design">
|
||
<title>Design</title>
|
||
<para>The Block Cache is an LRU cache that contains three levels of block priority to allow for scan-resistance and in-memory ColumnFamilies:
|
||
</para>
|
||
<itemizedlist>
|
||
<listitem>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.
|
||
</listitem>
|
||
<listitem>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.
|
||
</listitem>
|
||
<listitem>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.
|
||
</listitem>
|
||
</itemizedlist>
|
||
<para>
|
||
For more information, see the <link xlink:href="http://hbase.apache.org/xref/org/apache/hadoop/hbase/io/hfile/LruBlockCache.html">LruBlockCache source</link>
|
||
</para>
|
||
</section>
|
||
<section xml:id="block.cache.usage">
|
||
<title>Usage</title>
|
||
<para>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
|
||
<link xlink:href="http://en.wikipedia.org/wiki/Working_set_size">working set size</link>, 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.
|
||
</para>
|
||
<para>The way to calculate how much memory is available in HBase for caching is:
|
||
</para>
|
||
<programlisting>
|
||
number of region servers * heap size * hfile.block.cache.size * 0.85
|
||
</programlisting>
|
||
<para>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:
|
||
</para>
|
||
<itemizedlist>
|
||
<listitem>One region server with the default heap size (1GB) and the default block cache size will have 217MB of block cache available.
|
||
</listitem>
|
||
<listitem>20 region servers with the heap size set to 8GB and a default block cache size will have 34GB of block cache.
|
||
</listitem>
|
||
<listitem>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.
|
||
</listitem>
|
||
</itemizedlist>
|
||
<para>Your data isn't the only resident of the block cache, here are others that you may have to take into account:
|
||
</para>
|
||
<itemizedlist>
|
||
<listitem>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).
|
||
</listitem>
|
||
<listitem>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.
|
||
</listitem>
|
||
<listitem>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 <xref linkend="keysize"/>.
|
||
</listitem>
|
||
<listitem>Bloom filters: Just like the HFile indexes, those data structures (when enabled) are stored in the LRU.
|
||
</listitem>
|
||
</itemizedlist>
|
||
<para>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.
|
||
</para>
|
||
<para>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:
|
||
</para>
|
||
<itemizedlist>
|
||
<listitem>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 <xref linkend="trouble.log.gc"/>.
|
||
</listitem>
|
||
<listitem>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.
|
||
</listitem>
|
||
</itemizedlist>
|
||
</section>
|
||
</section>
|
||
|
||
<section xml:id="wal">
|
||
<title >Write Ahead Log (WAL)</title>
|
||
|
||
<section xml:id="purpose.wal">
|
||
<title>Purpose</title>
|
||
|
||
<para>Each RegionServer adds updates (Puts, Deletes) to its write-ahead log (WAL)
|
||
first, and then to the <xref linkend="store.memstore"/> for the affected <xref linkend="store" />.
|
||
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. <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/wal/HLog.html">HLog</link>
|
||
is the HBase WAL implementation, and there is one HLog instance per RegionServer.
|
||
</para>The WAL is in HDFS in <filename>/hbase/.logs/</filename> with subdirectories per region.
|
||
<para>
|
||
For more general information about the concept of write ahead logs, see the Wikipedia
|
||
<link xlink:href="http://en.wikipedia.org/wiki/Write-ahead_logging">Write-Ahead Log</link> article.
|
||
</para>
|
||
</section>
|
||
<section xml:id="wal_flush">
|
||
<title>WAL Flushing</title>
|
||
<para>TODO (describe).
|
||
</para>
|
||
</section>
|
||
|
||
<section xml:id="wal_splitting">
|
||
<title>WAL Splitting</title>
|
||
|
||
<section><title>How edits are recovered from a crashed RegionServer</title>
|
||
<para>When a RegionServer crashes, it will lose its ephemeral lease in
|
||
ZooKeeper...TODO</para>
|
||
</section>
|
||
<section>
|
||
<title><varname>hbase.hlog.split.skip.errors</varname></title>
|
||
|
||
<para>When set to <constant>true</constant>, the default, any error
|
||
encountered splitting will be logged, the problematic WAL will be
|
||
moved into the <filename>.corrupt</filename> directory under the hbase
|
||
<varname>rootdir</varname>, and processing will continue. If set to
|
||
<constant>false</constant>, the exception will be propagated and the
|
||
split logged as failed.<footnote>
|
||
<para>See <link
|
||
xlink:href="https://issues.apache.org/jira/browse/HBASE-2958">HBASE-2958
|
||
When hbase.hlog.split.skip.errors is set to false, we fail the
|
||
split but thats it</link>. We need to do more than just fail split
|
||
if this flag is set.</para>
|
||
</footnote></para>
|
||
</section>
|
||
|
||
<section>
|
||
<title>How EOFExceptions are treated when splitting a crashed
|
||
RegionServers' WALs</title>
|
||
|
||
<para>If we get an EOF while splitting logs, we proceed with the split
|
||
even when <varname>hbase.hlog.split.skip.errors</varname> ==
|
||
<constant>false</constant>. 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.<footnote>
|
||
<para>For background, see <link
|
||
xlink:href="https://issues.apache.org/jira/browse/HBASE-2643">HBASE-2643
|
||
Figure how to deal with eof splitting logs</link></para>
|
||
</footnote></para>
|
||
</section>
|
||
</section>
|
||
</section>
|
||
|
||
</section> <!-- regionserver -->
|
||
|
||
<section xml:id="regions.arch">
|
||
<title>Regions</title>
|
||
<para>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:
|
||
<programlisting>
|
||
<filename>Table</filename> (HBase table)
|
||
<filename>Region</filename> (Regions for the table)
|
||
<filename>Store</filename> (Store per ColumnFamily for each Region for the table)
|
||
<filename>MemStore</filename> (MemStore for each Store for each Region for the table)
|
||
<filename>StoreFile</filename> (StoreFiles for each Store for each Region for the table)
|
||
<filename>Block</filename> (Blocks within a StoreFile within a Store for each Region for the table)
|
||
</programlisting>
|
||
For a description of what HBase files look like when written to HDFS, see <xref linkend="trouble.namenode.hbase.objects"/>.
|
||
</para>
|
||
|
||
<section xml:id="arch.regions.size">
|
||
<title>Region Size</title>
|
||
|
||
<para>Determining the "right" region size can be tricky, and there are a few factors
|
||
to consider:</para>
|
||
|
||
<itemizedlist>
|
||
<listitem>
|
||
<para>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.</para>
|
||
</listitem>
|
||
|
||
<listitem>
|
||
<para>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.</para>
|
||
</listitem>
|
||
|
||
<listitem>
|
||
<para>There is not much memory footprint difference between 1 region
|
||
and 10 in terms of indexes, etc, held by the RegionServer.</para>
|
||
</listitem>
|
||
</itemizedlist>
|
||
|
||
<para>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).</para>
|
||
<para>See <xref linkend="bigger.regions"/> for more information on configuration.
|
||
</para>
|
||
</section>
|
||
|
||
<section xml:id="regions.arch.assignment">
|
||
<title>Region-RegionServer Assignment</title>
|
||
<para>This section describes how Regions are assigned to RegionServers.
|
||
</para>
|
||
|
||
<section xml:id="regions.arch.assignment.startup">
|
||
<title>Startup</title>
|
||
<para>When HBase starts regions are assigned as follows (short version):
|
||
<orderedlist>
|
||
<listitem>The Master invokes the <code>AssignmentManager</code> upon startup.
|
||
</listitem>
|
||
<listitem>The <code>AssignmentManager</code> looks at the existing region assignments in META.
|
||
</listitem>
|
||
<listitem>If the region assignment is still valid (i.e., if the RegionServer is still online)
|
||
then the assignment is kept.
|
||
</listitem>
|
||
<listitem>If the assignment is invalid, then the <code>LoadBalancerFactory</code> is invoked to assign the
|
||
region. The <code>DefaultLoadBalancer</code> will randomly assign the region to a RegionServer.
|
||
</listitem>
|
||
<listitem>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.
|
||
</listitem>
|
||
</orderedlist>
|
||
</para>
|
||
</section>
|
||
|
||
<section xml:id="regions.arch.assignment.failover">
|
||
<title>Failover</title>
|
||
<para>When a RegionServer fails (short version):
|
||
<orderedlist>
|
||
<listitem>The regions immediately become unavailable because the RegionServer is down.
|
||
</listitem>
|
||
<listitem>The Master will detect that the RegionServer has failed.
|
||
</listitem>
|
||
<listitem>The region assignments will be considered invalid and will be re-assigned just
|
||
like the startup sequence.
|
||
</listitem>
|
||
</orderedlist>
|
||
</para>
|
||
</section>
|
||
|
||
<section xml:id="regions.arch.balancer">
|
||
<title>Region Load Balancing</title>
|
||
<para>
|
||
Regions can be periodically moved by the <xref linkend="master.processes.loadbalancer" />.
|
||
</para>
|
||
</section>
|
||
|
||
</section> <!-- assignment -->
|
||
|
||
<section xml:id="regions.arch.locality">
|
||
<title>Region-RegionServer Locality</title>
|
||
<para>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:
|
||
<orderedlist>
|
||
<listitem>First replica is written to local node
|
||
</listitem>
|
||
<listitem>Second replica is written to another node in same rack
|
||
</listitem>
|
||
<listitem>Third replica is written to a node in another rack (if sufficient nodes)
|
||
</listitem>
|
||
</orderedlist>
|
||
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.
|
||
</para>
|
||
<para>For more information, see <link xlink:href="http://hadoop.apache.org/common/docs/r0.20.205.0/hdfs_design.html#Replica+Placement%3A+The+First+Baby+Steps">HDFS Design on Replica Placement</link>
|
||
and also Lars George's blog on <link xlink:href="http://www.larsgeorge.com/2010/05/hbase-file-locality-in-hdfs.html">HBase and HDFS locality</link>.
|
||
</para>
|
||
</section>
|
||
|
||
<section>
|
||
<title>Region Splits</title>
|
||
|
||
<para>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 <xref linkend="disable.splitting" /> for how to manually manage
|
||
splits (and for why you might do this)</para>
|
||
<section>
|
||
<title>Custom Split Policies</title>
|
||
<para>The default split policy can be overwritten using a custom <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.html">RegionSplitPolicy</link> (HBase 0.94+).
|
||
Typically a custom split policy should extend HBase's default split policy: <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.html">ConstantSizeRegionSplitPolicy</link>.
|
||
</para>
|
||
<para>The policy can set globally through the HBaseConfiguration used or on a per table basis:
|
||
<programlisting>
|
||
HTableDescriptor myHtd = ...;
|
||
myHtd.setValue(HTableDescriptor.SPLIT_POLICY, MyCustomSplitPolicy.class.getName());
|
||
</programlisting>
|
||
</para>
|
||
</section>
|
||
</section>
|
||
|
||
<section xml:id="store">
|
||
<title>Store</title>
|
||
<para>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.
|
||
</para>
|
||
<section xml:id="store.memstore">
|
||
<title>MemStore</title>
|
||
<para>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.</para>
|
||
</section>
|
||
<section xml:id="hfile">
|
||
<title>StoreFile (HFile)</title>
|
||
<para>StoreFiles are where your data lives.
|
||
</para>
|
||
<section><title>HFile Format</title>
|
||
<para>The <emphasis>hfile</emphasis> file format is based on
|
||
the SSTable file described in the <link xlink:href="http://research.google.com/archive/bigtable.html">BigTable [2006]</link> paper and on
|
||
Hadoop's <link xlink:href="http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/file/tfile/TFile.html">tfile</link>
|
||
(The unit test suite and the compression harness were taken directly from tfile).
|
||
Schubert Zhang's blog post on <link xlink:ref="http://cloudepr.blogspot.com/2009/09/hfile-block-indexed-file-format-to.html">HFile: A Block-Indexed File Format to Store Sorted Key-Value Pairs</link> makes for a thorough introduction to HBase's hfile. Matteo Bertozzi has also put up a
|
||
helpful description, <link xlink:href="http://th30z.blogspot.com/2011/02/hbase-io-hfile.html?spref=tw">HBase I/O: HFile</link>.
|
||
</para>
|
||
<para>For more information, see the <link xlink:href="http://hbase.apache.org/xref/org/apache/hadoop/hbase/io/hfile/HFile.html">HFile source code</link>.
|
||
Also see <xref linkend="hfilev2"/> for information about the HFile v2 format that was included in 0.92.
|
||
</para>
|
||
</section>
|
||
<section xml:id="hfile_tool">
|
||
<title>HFile Tool</title>
|
||
|
||
<para>To view a textualized version of hfile content, you can do use
|
||
the <classname>org.apache.hadoop.hbase.io.hfile.HFile
|
||
</classname>tool. Type the following to see usage:<programlisting><code>$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.io.hfile.HFile </code> </programlisting>For
|
||
example, to view the content of the file
|
||
<filename>hdfs://10.81.47.41:8020/hbase/TEST/1418428042/DSMP/4759508618286845475</filename>,
|
||
type the following:<programlisting> <code>$ ${HBASE_HOME}/bin/hbase org.apache.hadoop.hbase.io.hfile.HFile -v -f hdfs://10.81.47.41:8020/hbase/TEST/1418428042/DSMP/4759508618286845475 </code> </programlisting>If
|
||
you leave off the option -v to see just a summary on the hfile. See
|
||
usage for other things to do with the <classname>HFile</classname>
|
||
tool.</para>
|
||
</section>
|
||
<section xml:id="store.file.dir">
|
||
<title>StoreFile Directory Structure on HDFS</title>
|
||
<para>For more information of what StoreFiles look like on HDFS with respect to the directory structure, see <xref linkend="trouble.namenode.hbase.objects" />.
|
||
</para>
|
||
</section>
|
||
</section> <!-- hfile -->
|
||
|
||
<section xml:id="hfile.blocks">
|
||
<title>Blocks</title>
|
||
<para>StoreFiles are composed of blocks. The blocksize is configured on a per-ColumnFamily basis.
|
||
</para>
|
||
<para>Compression happens at the block level within StoreFiles. For more information on compression, see <xref linkend="compression"/>.
|
||
</para>
|
||
<para>For more information on blocks, see the <link xlink:href="http://hbase.apache.org/xref/org/apache/hadoop/hbase/io/hfile/HFileBlock.html">HFileBlock source code</link>.
|
||
</para>
|
||
</section>
|
||
<section xml:id="keyvalue">
|
||
<title>KeyValue</title>
|
||
<para>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.
|
||
</para>
|
||
<para>The KeyValue format inside a byte array is:
|
||
<itemizedlist>
|
||
<listitem>keylength</listitem>
|
||
<listitem>valuelength</listitem>
|
||
<listitem>key</listitem>
|
||
<listitem>value</listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
<para>The Key is further decomposed as:
|
||
<itemizedlist>
|
||
<listitem>rowlength</listitem>
|
||
<listitem>row (i.e., the rowkey)</listitem>
|
||
<listitem>columnfamilylength</listitem>
|
||
<listitem>columnfamily</listitem>
|
||
<listitem>columnqualifier</listitem>
|
||
<listitem>timestamp</listitem>
|
||
<listitem>keytype (e.g., Put, Delete, DeleteColumn, DeleteFamily)</listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
<para>KeyValue instances are <emphasis>not</emphasis> 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 <link xlink:href="http://hbase.apache.org/xref/org/apache/hadoop/hbase/KeyValue.html">KeyValue source code</link>.
|
||
</para>
|
||
<section xml:id="keyvalue.example"><title>Example</title>
|
||
<para>To emphasize the points above, examine what happens with two Puts for two different columns for the same row:</para>
|
||
<itemizedlist>
|
||
<listitem>Put #1: <code>rowkey=row1, cf:attr1=value1</code></listitem>
|
||
<listitem>Put #2: <code>rowkey=row1, cf:attr2=value2</code></listitem>
|
||
</itemizedlist>
|
||
<para>Even though these are for the same row, a KeyValue is created for each column:</para>
|
||
<para>Key portion for Put #1:
|
||
<itemizedlist>
|
||
<listitem>rowlength <code>------------> 4</code></listitem>
|
||
<listitem>row <code>-----------------> row1</code></listitem>
|
||
<listitem>columnfamilylength <code>---> 2</code></listitem>
|
||
<listitem>columnfamily <code>--------> cf</code></listitem>
|
||
<listitem>columnqualifier <code>------> attr1</code></listitem>
|
||
<listitem>timestamp <code>-----------> server time of Put</code></listitem>
|
||
<listitem>keytype <code>-------------> Put</code></listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
<para>Key portion for Put #2:
|
||
<itemizedlist>
|
||
<listitem>rowlength <code>------------> 4</code></listitem>
|
||
<listitem>row <code>-----------------> row1</code></listitem>
|
||
<listitem>columnfamilylength <code>---> 2</code></listitem>
|
||
<listitem>columnfamily <code>--------> cf</code></listitem>
|
||
<listitem>columnqualifier <code>------> attr2</code></listitem>
|
||
<listitem>timestamp <code>-----------> server time of Put</code></listitem>
|
||
<listitem>keytype <code>-------------> Put</code></listitem>
|
||
</itemizedlist>
|
||
<!-- trying to get the above arrows to line up at the same horizontal position has proven
|
||
harder than I thought -->
|
||
</para>
|
||
</section>
|
||
<para>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.</para>
|
||
</section>
|
||
<section xml:id="compaction">
|
||
<title>Compaction</title>
|
||
<para>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.
|
||
</para>
|
||
<para>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 <xref linkend="managed.compactions" />.
|
||
</para>
|
||
<para>Compactions will <emphasis>not</emphasis> perform region merges. See <xref linkend="ops.regionmgt.merge"/> for more information on region merging.
|
||
</para>
|
||
<section xml:id="compaction.file.selection">
|
||
<title>Compaction File Selection</title>
|
||
<para>To understand the core algorithm for StoreFile selection, there is some ASCII-art in the <link xlink:href="http://hbase.apache.org/xref/org/apache/hadoop/hbase/regionserver/Store.html#836">Store source code</link> that
|
||
will serve as useful reference. It has been copied below:
|
||
<programlisting>
|
||
/* normal skew:
|
||
*
|
||
* older ----> newer
|
||
* _
|
||
* | | _
|
||
* | | | | _
|
||
* --|-|- |-|- |-|---_-------_------- minCompactSize
|
||
* | | | | | | | | _ | |
|
||
* | | | | | | | | | | | |
|
||
* | | | | | | | | | | | |
|
||
*/
|
||
</programlisting>
|
||
Important knobs:
|
||
<itemizedlist>
|
||
<listitem><code>hbase.store.compaction.ratio</code> Ratio used in compaction
|
||
file selection algorithm (default 1.2f). </listitem>
|
||
<listitem><code>hbase.hstore.compaction.min</code> (.90 hbase.hstore.compactionThreshold) (files) Minimum number
|
||
of StoreFiles per Store to be selected for a compaction to occur (default 2).</listitem>
|
||
<listitem><code>hbase.hstore.compaction.max</code> (files) Maximum number of StoreFiles to compact per minor compaction (default 10).</listitem>
|
||
<listitem><code>hbase.hstore.compaction.min.size</code> (bytes)
|
||
Any StoreFile smaller than this setting with automatically be a candidate for compaction. Defaults to
|
||
<code>hbase.hregion.memstore.flush.size</code> (128 mb). </listitem>
|
||
<listitem><code>hbase.hstore.compaction.max.size</code> (.92) (bytes)
|
||
Any StoreFile larger than this setting with automatically be excluded from compaction (default Long.MAX_VALUE). </listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
<para>The minor compaction StoreFile selection logic is size based, and selects a file for compaction when the file
|
||
<= sum(smaller_files) * <code>hbase.hstore.compaction.ratio</code>.
|
||
</para>
|
||
</section>
|
||
<section xml:id="compaction.file.selection.example1">
|
||
<title>Minor Compaction File Selection - Example #1 (Basic Example)</title>
|
||
<para>This example mirrors an example from the unit test <code>TestCompactSelection</code>.
|
||
<itemizedlist>
|
||
<listitem><code>hbase.store.compaction.ratio</code> = 1.0f </listitem>
|
||
<listitem><code>hbase.hstore.compaction.min</code> = 3 (files) </listitem>>
|
||
<listitem><code>hbase.hstore.compaction.max</code> = 5 (files) </listitem>>
|
||
<listitem><code>hbase.hstore.compaction.min.size</code> = 10 (bytes) </listitem>>
|
||
<listitem><code>hbase.hstore.compaction.max.size</code> = 1000 (bytes) </listitem>>
|
||
</itemizedlist>
|
||
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.
|
||
</para>
|
||
<para>Why?
|
||
<itemizedlist>
|
||
<listitem>100 --> No, because sum(50, 23, 12, 12) * 1.0 = 97. </listitem>
|
||
<listitem>50 --> No, because sum(23, 12, 12) * 1.0 = 47. </listitem>
|
||
<listitem>23 --> Yes, because sum(12, 12) * 1.0 = 24. </listitem>
|
||
<listitem>12 --> Yes, because the previous file has been included, and because this
|
||
does not exceed the the max-file limit of 5 </listitem>
|
||
<listitem>12 --> Yes, because the previous file had been included, and because this
|
||
does not exceed the the max-file limit of 5.</listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
</section>
|
||
<section xml:id="compaction.file.selection.example2">
|
||
<title>Minor Compaction File Selection - Example #2 (Not Enough Files To Compact)</title>
|
||
<para>This example mirrors an example from the unit test <code>TestCompactSelection</code>.
|
||
<itemizedlist>
|
||
<listitem><code>hbase.store.compaction.ratio</code> = 1.0f </listitem>
|
||
<listitem><code>hbase.hstore.compaction.min</code> = 3 (files) </listitem>>
|
||
<listitem><code>hbase.hstore.compaction.max</code> = 5 (files) </listitem>>
|
||
<listitem><code>hbase.hstore.compaction.min.size</code> = 10 (bytes) </listitem>>
|
||
<listitem><code>hbase.hstore.compaction.max.size</code> = 1000 (bytes) </listitem>>
|
||
</itemizedlist>
|
||
</para>
|
||
<para>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.
|
||
</para>
|
||
<para>Why?
|
||
<itemizedlist>
|
||
<listitem>100 --> No, because sum(25, 12, 12) * 1.0 = 47</listitem>
|
||
<listitem>25 --> No, because sum(12, 12) * 1.0 = 24</listitem>
|
||
<listitem>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</listitem>
|
||
<listitem>12 --> No. Candidate because the previous StoreFile was, but there are not enough files to compact</listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
</section>
|
||
<section xml:id="compaction.file.selection.example2">
|
||
<title>Minor Compaction File Selection - Example #3 (Limiting Files To Compact)</title>
|
||
<para>This example mirrors an example from the unit test <code>TestCompactSelection</code>.
|
||
<itemizedlist>
|
||
<listitem><code>hbase.store.compaction.ratio</code> = 1.0f </listitem>
|
||
<listitem><code>hbase.hstore.compaction.min</code> = 3 (files) </listitem>>
|
||
<listitem><code>hbase.hstore.compaction.max</code> = 5 (files) </listitem>>
|
||
<listitem><code>hbase.hstore.compaction.min.size</code> = 10 (bytes) </listitem>>
|
||
<listitem><code>hbase.hstore.compaction.max.size</code> = 1000 (bytes) </listitem>>
|
||
</itemizedlist>
|
||
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.
|
||
</para>
|
||
<para>Why?
|
||
<itemizedlist>
|
||
<listitem>7 --> Yes, because sum(6, 5, 4, 3, 2, 1) * 1.0 = 21. Also, 7 is less than the min-size</listitem>
|
||
<listitem>6 --> Yes, because sum(5, 4, 3, 2, 1) * 1.0 = 15. Also, 6 is less than the min-size. </listitem>
|
||
<listitem>5 --> Yes, because sum(4, 3, 2, 1) * 1.0 = 10. Also, 5 is less than the min-size. </listitem>
|
||
<listitem>4 --> Yes, because sum(3, 2, 1) * 1.0 = 6. Also, 4 is less than the min-size. </listitem>
|
||
<listitem>3 --> Yes, because sum(2, 1) * 1.0 = 3. Also, 3 is less than the min-size. </listitem>
|
||
<listitem>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. </listitem>
|
||
<listitem>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. </listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
</section>
|
||
<section xml:id="compaction.config.impact">
|
||
<title>Impact of Key Configuration Options</title>
|
||
<para><code>hbase.store.compaction.ratio</code>. 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.
|
||
</para>
|
||
<para><code>hbase.hstore.compaction.min.size</code>. 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.
|
||
</para>
|
||
</section>
|
||
</section> <!-- compaction -->
|
||
|
||
</section> <!-- store -->
|
||
|
||
<section xml:id="blooms">
|
||
<title>Bloom Filters</title>
|
||
<para><link xlink:href="http://en.wikipedia.org/wiki/Bloom_filter">Bloom filters</link> were developed over in <link
|
||
xlink:href="https://issues.apache.org/jira/browse/HBASE-1200">HBase-1200
|
||
Add bloomfilters</link>.<footnote>
|
||
<para>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 <emphasis>Development Process</emphasis> section
|
||
of the document <link
|
||
xlink:href="https://issues.apache.org/jira/secure/attachment/12444007/Bloom_Filters_in_HBase.pdf">BloomFilters
|
||
in HBase</link> attached to <link
|
||
xlink:href="https://issues.apache.org/jira/browse/HBASE-1200">HBase-1200</link>.</para>
|
||
</footnote><footnote>
|
||
<para>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 <link
|
||
xlink:href="http://www.one-lab.org">European Commission One-Lab
|
||
Project 034819</link>. 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.</para>
|
||
</footnote></para>
|
||
<para>See also <xref linkend="schema.bloom" /> and <xref linkend="config.bloom" />.
|
||
</para>
|
||
|
||
<section xml:id="bloom_footprint">
|
||
<title>Bloom StoreFile footprint</title>
|
||
|
||
<para>Bloom filters add an entry to the <classname>StoreFile</classname>
|
||
general <classname>FileInfo</classname> data structure and then two
|
||
extra entries to the <classname>StoreFile</classname> metadata
|
||
section.</para>
|
||
|
||
<section>
|
||
<title>BloomFilter in the <classname>StoreFile</classname>
|
||
<classname>FileInfo</classname> data structure</title>
|
||
|
||
<para><classname>FileInfo</classname> has a
|
||
<varname>BLOOM_FILTER_TYPE</varname> entry which is set to
|
||
<varname>NONE</varname>, <varname>ROW</varname> or
|
||
<varname>ROWCOL.</varname></para>
|
||
</section>
|
||
|
||
<section>
|
||
<title>BloomFilter entries in <classname>StoreFile</classname>
|
||
metadata</title>
|
||
|
||
<para><varname>BLOOM_FILTER_META</varname> holds Bloom Size, Hash
|
||
Function used, etc. Its small in size and is cached on
|
||
<classname>StoreFile.Reader</classname> load</para>
|
||
<para><varname>BLOOM_FILTER_DATA</varname> is the actual bloomfilter
|
||
data. Obtained on-demand. Stored in the LRU cache, if it is enabled
|
||
(Its enabled by default).</para>
|
||
</section>
|
||
</section>
|
||
</section> <!-- bloom -->
|
||
|
||
</section> <!-- regions -->
|
||
|
||
<section xml:id="arch.bulk.load"><title>Bulk Loading</title>
|
||
<section xml:id="arch.bulk.load.overview"><title>Overview</title>
|
||
<para>
|
||
HBase includes several methods of loading data into tables.
|
||
The most straightforward method is to either use the <code>TableOutputFormat</code>
|
||
class from a MapReduce job, or use the normal client APIs; however,
|
||
these are not always the most efficient methods.
|
||
</para>
|
||
<para>
|
||
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.
|
||
</para>
|
||
</section>
|
||
<section xml:id="arch.bulk.load.arch"><title>Bulk Load Architecture</title>
|
||
<para>
|
||
The HBase bulk load process consists of two main steps.
|
||
</para>
|
||
<section xml:id="arch.bulk.load.prep"><title>Preparing data via a MapReduce job</title>
|
||
<para>
|
||
The first step of a bulk load is to generate HBase data files (StoreFiles) from
|
||
a MapReduce job using <code>HFileOutputFormat</code>. This output format writes
|
||
out data in HBase's internal storage format so that they can be
|
||
later loaded very efficiently into the cluster.
|
||
</para>
|
||
<para>
|
||
In order to function efficiently, <code>HFileOutputFormat</code> 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 <code>TotalOrderPartitioner</code> class to partition the map output
|
||
into disjoint ranges of the key space, corresponding to the key
|
||
ranges of the regions in the table.
|
||
</para>
|
||
<para>
|
||
<code>HFileOutputFormat</code> includes a convenience function,
|
||
<code>configureIncrementalLoad()</code>, which automatically sets up
|
||
a <code>TotalOrderPartitioner</code> based on the current region boundaries of a
|
||
table.
|
||
</para>
|
||
</section>
|
||
<section xml:id="arch.bulk.load.complete"><title>Completing the data load</title>
|
||
<para>
|
||
After the data has been prepared using
|
||
<code>HFileOutputFormat</code>, it is loaded into the cluster using
|
||
<code>completebulkload</code>. 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.
|
||
</para>
|
||
<para>
|
||
If the region boundaries have changed during the course of bulk load
|
||
preparation, or between the preparation and completion steps, the
|
||
<code>completebulkloads</code> 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.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
<section xml:id="arch.bulk.load.import"><title>Importing the prepared data using the completebulkload tool</title>
|
||
<para>
|
||
After a data import has been prepared, either by using the
|
||
<code>importtsv</code> tool with the
|
||
"<code>importtsv.bulk.output</code>" option or by some other MapReduce
|
||
job using the <code>HFileOutputFormat</code>, the
|
||
<code>completebulkload</code> tool is used to import the data into the
|
||
running cluster.
|
||
</para>
|
||
<para>
|
||
The <code>completebulkload</code> tool simply takes the output path
|
||
where <code>importtsv</code> or your MapReduce job put its results, and
|
||
the table name to import into. For example:
|
||
</para>
|
||
<code>$ hadoop jar hbase-VERSION.jar completebulkload [-c /path/to/hbase/config/hbase-site.xml] /user/todd/myoutput mytable</code>
|
||
<para>
|
||
The <code>-c config-file</code> 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).
|
||
</para>
|
||
<para>
|
||
Note: If the target table does not already exist in HBase, this
|
||
tool will create the table automatically.</para>
|
||
<para>
|
||
This tool will run quickly, after which point the new data will be visible in
|
||
the cluster.
|
||
</para>
|
||
</section>
|
||
<section xml:id="arch.bulk.load.also"><title>See Also</title>
|
||
<para>For more information about the referenced utilities, see <xref linkend="importtsv"/> and <xref linkend="completebulkload"/>.
|
||
</para>
|
||
</section>
|
||
<section xml:id="arch.bulk.load.adv"><title>Advanced Usage</title>
|
||
<para>
|
||
Although the <code>importtsv</code> 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 <code>ImportTsv.java</code> and check the JavaDoc for
|
||
HFileOutputFormat.
|
||
</para>
|
||
<para>
|
||
The import step of the bulk load can also be done programatically. See the
|
||
<code>LoadIncrementalHFiles</code> class for more information.
|
||
</para>
|
||
</section>
|
||
</section> <!-- bulk loading -->
|
||
|
||
<section xml:id="arch.hdfs"><title>HDFS</title>
|
||
<para>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.
|
||
</para>
|
||
<para>See the Hadoop documentation on <link xlink:href="http://hadoop.apache.org/common/docs/current/hdfs_design.html">HDFS Architecture</link>
|
||
for more information.
|
||
</para>
|
||
<section xml:id="arch.hdfs.nn"><title>NameNode</title>
|
||
<para>The NameNode is responsible for maintaining the filesystem metadata. See the above HDFS Architecture link
|
||
for more information.
|
||
</para>
|
||
</section>
|
||
<section xml:id="arch.hdfs.dn"><title>DataNode</title>
|
||
<para>The DataNodes are responsible for storing HDFS blocks. See the above HDFS Architecture link
|
||
for more information.
|
||
</para>
|
||
</section>
|
||
</section>
|
||
|
||
</chapter> <!-- architecture -->
|
||
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="external_apis.xml" />
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="performance.xml" />
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="troubleshooting.xml" />
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="case_studies.xml" />
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="ops_mgt.xml" />
|
||
<xi:include xmlns:xi="http://www.w3.org/2001/XInclude" href="developer.xml" />
|
||
|
||
<appendix xml:id="faq">
|
||
<title >FAQ</title>
|
||
<qandaset defaultlabel='faq'>
|
||
<qandadiv><title>General</title>
|
||
<qandaentry>
|
||
<question><para>When should I use HBase?</para></question>
|
||
<answer>
|
||
<para>See the <xref linkend="arch.overview" /> in the Architecture chapter.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
<qandaentry>
|
||
<question><para>Are there other HBase FAQs?</para></question>
|
||
<answer>
|
||
<para>
|
||
See the FAQ that is up on the wiki, <link xlink:href="http://wiki.apache.org/hadoop/Hbase/FAQ">HBase Wiki FAQ</link>.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
<qandaentry xml:id="faq.sql">
|
||
<question><para>Does HBase support SQL?</para></question>
|
||
<answer>
|
||
<para>
|
||
Not really. SQL-ish support for HBase via <link xlink:href="http://hive.apache.org/">Hive</link> is in development, however Hive is based on MapReduce which is not generally suitable for low-latency requests.
|
||
See the <xref linkend="datamodel" /> section for examples on the HBase client.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
<qandaentry>
|
||
<question><para>How can I find examples of NoSQL/HBase?</para></question>
|
||
<answer>
|
||
<para>See the link to the BigTable paper in <xref linkend="other.info" /> in the appendix, as
|
||
well as the other papers.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
<qandaentry>
|
||
<question><para>What is the history of HBase?</para></question>
|
||
<answer>
|
||
<para>See <xref linkend="hbase.history"/>.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
</qandadiv>
|
||
<qandadiv xml:id="faq.arch"><title>Architecture</title>
|
||
<qandaentry xml:id="faq.arch.regions">
|
||
<question><para>How does HBase handle Region-RegionServer assignment and locality?</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="regions.arch" />.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
</qandadiv>
|
||
<qandadiv xml:id="faq.config"><title>Configuration</title>
|
||
<qandaentry xml:id="faq.config.started">
|
||
<question><para>How can I get started with my first cluster?</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="quickstart" />.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
<qandaentry xml:id="faq.config.started">
|
||
<question><para>Where can I learn about the rest of the configuration options?</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="configuration" />.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
</qandadiv>
|
||
<qandadiv xml:id="faq.design"><title>Schema Design / Data Access</title>
|
||
<qandaentry xml:id="faq.design.schema">
|
||
<question><para>How should I design my schema in HBase?</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="datamodel" /> and <xref linkend="schema" />
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
<qandaentry>
|
||
<question><para>
|
||
How can I store (fill in the blank) in HBase?
|
||
</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="supported.datatypes" />.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
<qandaentry xml:id="secondary.indices">
|
||
<question><para>
|
||
How can I handle secondary indexes in HBase?
|
||
</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="secondary.indexes" />
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
<qandaentry xml:id="faq.changing.rowkeys">
|
||
<question><para>Can I change a table's rowkeys?</para></question>
|
||
<answer>
|
||
<para>
|
||
This is a very common quesiton. You can't. See <xref linkend="changing.rowkeys" />.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
<qandaentry xml:id="faq.apis">
|
||
<question><para>What APIs does HBase support?</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="datamodel" />, <xref linkend="client" /> and <xref linkend="nonjava.jvm"/>.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
</qandadiv>
|
||
<qandadiv xml:id="faq.mapreduce"><title>MapReduce</title>
|
||
<qandaentry xml:id="faq.mapreduce.use">
|
||
<question><para>How can I use MapReduce with HBase?</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="mapreduce" />
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
</qandadiv>
|
||
<qandadiv><title>Performance and Troubleshooting</title>
|
||
<qandaentry>
|
||
<question><para>
|
||
How can I improve HBase cluster performance?
|
||
</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="performance" />.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
<qandaentry>
|
||
<question><para>
|
||
How can I troubleshoot my HBase cluster?
|
||
</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="trouble" />.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
</qandadiv>
|
||
<qandadiv xml:id="ec2"><title>Amazon EC2</title>
|
||
<qandaentry>
|
||
<question><para>
|
||
I am running HBase on Amazon EC2 and...
|
||
</para></question>
|
||
<answer>
|
||
<para>
|
||
EC2 issues are a special case. See Troubleshooting <xref linkend="trouble.ec2" /> and Performance <xref linkend="perf.ec2" /> sections.
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
</qandadiv>
|
||
<qandadiv><title xml:id="faq.operations">Operations</title>
|
||
<qandaentry>
|
||
<question><para>
|
||
How do I manage my HBase cluster?
|
||
</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="ops_mgt" />
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
<qandaentry>
|
||
<question><para>
|
||
How do I back up my HBase cluster?
|
||
</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="ops.backup" />
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
</qandadiv>
|
||
<qandadiv><title>HBase in Action</title>
|
||
<qandaentry>
|
||
<question><para>Where can I find interesting videos and presentations on HBase?</para></question>
|
||
<answer>
|
||
<para>
|
||
See <xref linkend="other.info" />
|
||
</para>
|
||
</answer>
|
||
</qandaentry>
|
||
</qandadiv>
|
||
</qandaset>
|
||
</appendix>
|
||
|
||
<appendix xml:id="hbck.in.depth">
|
||
<title>hbck In Depth</title>
|
||
<para>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.
|
||
</para>
|
||
<section>
|
||
<title>Running hbck to identify inconsistencies</title>
|
||
To check to see if your HBase cluster has corruptions, run hbck against your HBase cluster:
|
||
<programlisting>
|
||
$ ./bin/hbase hbck
|
||
</programlisting>
|
||
<para>
|
||
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 <code>-details</code> option will report more details including a representative
|
||
listing of all the splits present in all the tables.
|
||
</para>
|
||
<programlisting>
|
||
$ ./bin/hbase hbck -details
|
||
</programlisting>
|
||
</section>
|
||
<section><title>Inconsistencies</title>
|
||
<para>
|
||
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.
|
||
</para>
|
||
<para>
|
||
There are two invariants that when violated create inconsistencies in HBase:
|
||
</para>
|
||
<itemizedlist>
|
||
<listitem>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.
|
||
</listitem>
|
||
<listitem>HBase’s table integrity invariant is satisfied if for each table, every possible row key
|
||
resolves to exactly one region.
|
||
</listitem>
|
||
</itemizedlist>
|
||
<para>
|
||
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 <code>-fix</code> 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.
|
||
</para>
|
||
<para>
|
||
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.
|
||
</para>
|
||
</section>
|
||
<section><title>Localized repairs</title>
|
||
<para>
|
||
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:
|
||
<itemizedlist>
|
||
<listitem><code>-fixAssignments</code> (equivalent to the 0.90 <code>-fix</code> option) repairs unassigned, incorrectly
|
||
assigned or multiply assigned regions.
|
||
</listitem>
|
||
<listitem><code>-fixMeta</code> 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.
|
||
</listitem>
|
||
</itemizedlist>
|
||
To fix deployment and assignment problems you can run this command:
|
||
</para>
|
||
<programlisting>
|
||
$ ./bin/hbase hbck -fixAssignments
|
||
</programlisting>
|
||
To fix deployment and assignment problems as well as repairing incorrect meta rows you can
|
||
run this command:.
|
||
<programlisting>
|
||
$ ./bin/hbase hbck -fixAssignments -fixMeta
|
||
</programlisting>
|
||
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:
|
||
<itemizedlist>
|
||
<listitem><code>-fixHdfsHoles</code> 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.
|
||
</listitem>
|
||
</itemizedlist>
|
||
<programlisting>
|
||
$ ./bin/hbase hbck -fixAssignments -fixMeta -fixHdfsHoles
|
||
</programlisting>
|
||
Since this is a common operation, we’ve added a the <code>-repairHoles</code> flag that is equivalent to the
|
||
previous command:
|
||
<programlisting>
|
||
$ ./bin/hbase hbck -repairHoles
|
||
</programlisting>
|
||
If inconsistencies still remain after these steps, you most likely have table integrity problems
|
||
related to orphaned or overlapping regions.
|
||
</section>
|
||
<section><title>Region Overlap Repairs</title>
|
||
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 <code>hbck -details</code>
|
||
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:
|
||
<itemizedlist>
|
||
<listitem><code>-fixHdfsOrphans</code> option for “adopting” a region directory that is missing a region
|
||
metadata file (the .regioninfo file).
|
||
</listitem>
|
||
<listitem><code>-fixHdfsOverlaps</code> ability for fixing overlapping regions
|
||
</listitem>
|
||
</itemizedlist>
|
||
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.
|
||
<itemizedlist>
|
||
<listitem><code>-maxMerge <n></code> maximum number of overlapping regions to merge
|
||
</listitem>
|
||
<listitem><code>-sidelineBigOverlaps</code> if more than maxMerge regions are overlapping, sideline attempt
|
||
to sideline the regions overlapping with the most other regions.
|
||
</listitem>
|
||
<listitem><code>-maxOverlapsToSideline <n></code> if sidelining large overlapping regions, sideline at most n
|
||
regions.
|
||
</listitem>
|
||
</itemizedlist>
|
||
|
||
Since often times you would just want to get the tables repaired, you can use this option to turn
|
||
on all repair options:
|
||
<itemizedlist>
|
||
<listitem><code>-repair</code> includes all the region consistency options and only the hole repairing table
|
||
integrity options.
|
||
</listitem>
|
||
</itemizedlist>
|
||
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.
|
||
<programlisting>
|
||
$ ./bin/hbase/ hbck -repair TableFoo TableBar
|
||
</programlisting>
|
||
<section><title>Special cases: Meta is not properly assigned</title>
|
||
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 <code>-fixMetaOnly</code> option that can try to fix meta assignments.
|
||
<programlisting>
|
||
$ ./bin/hbase hbck -fixMetaOnly -fixAssignments
|
||
</programlisting>
|
||
</section>
|
||
<section><title>Special cases: HBase version file is missing</title>
|
||
HBase’s data on the file system requires a version file in order to start. If this flie is missing, you
|
||
can use the <code>-fixVersionFile</code> 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.
|
||
</section>
|
||
<section><title>Special case: Root and META are corrupt.</title>
|
||
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.
|
||
<programlisting>
|
||
$ ./bin/hbase org.apache.hadoop.hbase.util.OfflineMetaRepair
|
||
</programlisting>
|
||
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.
|
||
</section>
|
||
</section>
|
||
</appendix>
|
||
|
||
<appendix xml:id="compression">
|
||
|
||
<title >Compression In HBase<indexterm><primary>Compression</primary></indexterm></title>
|
||
|
||
<section xml:id="compression.test">
|
||
<title>CompressionTest Tool</title>
|
||
<para>
|
||
HBase includes a tool to test compression is set up properly.
|
||
To run it, type <code>/bin/hbase org.apache.hadoop.hbase.util.CompressionTest</code>.
|
||
This will emit usage on how to run the tool.
|
||
</para>
|
||
</section>
|
||
|
||
<section xml:id="hbase.regionserver.codecs">
|
||
<title>
|
||
<varname>
|
||
hbase.regionserver.codecs
|
||
</varname>
|
||
</title>
|
||
<para>
|
||
To have a RegionServer test a set of codecs and fail-to-start if any
|
||
code is missing or misinstalled, add the configuration
|
||
<varname>
|
||
hbase.regionserver.codecs
|
||
</varname>
|
||
to your <filename>hbase-site.xml</filename> with a value of
|
||
codecs to test on startup. For example if the
|
||
<varname>
|
||
hbase.regionserver.codecs
|
||
</varname> value is <code>lzo,gz</code> and if lzo is not present
|
||
or improperly installed, the misconfigured RegionServer will fail
|
||
to start.
|
||
</para>
|
||
<para>
|
||
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.
|
||
</para>
|
||
</section>
|
||
|
||
<section xml:id="lzo.compression">
|
||
<title>
|
||
LZO
|
||
</title>
|
||
<para>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 <link xlink:href="http://wiki.apache.org/hadoop/UsingLzoCompression">Using LZO Compression</link>
|
||
wiki page for how to make LZO work with HBase.
|
||
</para>
|
||
<para>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. </para>
|
||
<para>See <xref linkend="hbase.regionserver.codecs" />
|
||
for a feature to help protect against failed LZO install.</para>
|
||
</section>
|
||
|
||
<section xml:id="gzip.compression">
|
||
<title>
|
||
GZIP
|
||
</title>
|
||
<para>
|
||
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 <emphasis>Got brand-new compressor</emphasis>
|
||
reports in your logs; see <xref linkend="brand.new.compressor" />).
|
||
</para>
|
||
</section>
|
||
<section xml:id="snappy.compression">
|
||
<title>
|
||
SNAPPY
|
||
</title>
|
||
<para>
|
||
If snappy is installed, HBase can make use of it (courtesy of
|
||
<link xlink:href="http://code.google.com/p/hadoop-snappy/">hadoop-snappy</link>
|
||
<footnote><para>See <link xlink:href="http://search-hadoop.com/m/Ds8d51c263B1/%2522Hadoop-Snappy+in+synch+with+Hadoop+trunk%2522&subj=Hadoop+Snappy+in+synch+with+Hadoop+trunk">Alejandro's note</link> up on the list on difference between Snappy in Hadoop
|
||
and Snappy in HBase</para></footnote>).
|
||
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>
|
||
Build and install <link xlink:href="http://code.google.com/p/snappy/">snappy</link> on all nodes
|
||
of your cluster.
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
Use CompressionTest to verify snappy support is enabled and the libs can be loaded ON ALL NODES of your cluster:
|
||
<programlisting>$ hbase org.apache.hadoop.hbase.util.CompressionTest hdfs://host/path/to/hbase snappy</programlisting>
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
Create a column family with snappy compression and verify it in the hbase shell:
|
||
<programlisting>$ hbase> create 't1', { NAME => 'cf1', COMPRESSION => 'SNAPPY' }
|
||
hbase> describe 't1'</programlisting>
|
||
In the output of the "describe" command, you need to ensure it lists "COMPRESSION => 'SNAPPY'"
|
||
</para>
|
||
</listitem>
|
||
|
||
</orderedlist>
|
||
|
||
</para>
|
||
</section>
|
||
<section xml:id="changing.compression">
|
||
<title>Changing Compression Schemes</title>
|
||
<para>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
|
||
<emphasis>not</emphasis> need to be re-created and the data does <emphasis>not</emphasis> 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.
|
||
</para>
|
||
</section>
|
||
</appendix>
|
||
|
||
<appendix>
|
||
<title xml:id="ycsb"><link xlink:href="https://github.com/brianfrankcooper/YCSB/">YCSB: The Yahoo! Cloud Serving Benchmark</link> and HBase</title>
|
||
<para>TODO: Describe how YCSB is poor for putting up a decent cluster load.</para>
|
||
<para>TODO: Describe setup of YCSB for HBase</para>
|
||
<para>Ted Dunning redid YCSB so its mavenized and added facility for verifying workloads. See <link xlink:href="https://github.com/tdunning/YCSB">Ted Dunning's YCSB</link>.</para>
|
||
|
||
</appendix>
|
||
|
||
<appendix xml:id="hfilev2">
|
||
<title>HFile format version 2</title>
|
||
|
||
<section><title>Motivation </title>
|
||
<para>Note: this feature was introduced in HBase 0.92</para>
|
||
<para>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.</para>
|
||
<para>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.</para>
|
||
<para>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. </para>
|
||
<para>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.</para></section>
|
||
<section><title>HFile format version 1 overview </title><para>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:
|
||
<inlinemediaobject>
|
||
<imageobject>
|
||
<imagedata align="middle" valign="middle" fileref="hfile.png" />
|
||
</imageobject>
|
||
<textobject>
|
||
<phrase>HFile Version 1</phrase>
|
||
</textobject>
|
||
<caption>
|
||
<para>HFile Version 1
|
||
</para>
|
||
</caption>
|
||
</inlinemediaobject>
|
||
<footnote><para>Image courtesy of Lars George, <link xlink:href="http://www.larsgeorge.com/2009/10/hbase-architecture-101-storage.html">hbase-architecture-101-storage.html</link>.</para></footnote>
|
||
</para>
|
||
<section><title> Block index format in version 1 </title>
|
||
<para>The block index in version 1 is very straightforward. For each entry, it contains: </para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>Offset (long)</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Uncompressed size (int)</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Key (a serialized byte array written using Bytes.writeByteArray) </para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>Key length as a variable-length integer (VInt)
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
Key bytes
|
||
</para>
|
||
</listitem>
|
||
</orderedlist>
|
||
</listitem>
|
||
</orderedlist>
|
||
<para>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.</para></section></section><section><title>
|
||
HBase file format with inline blocks (version 2)
|
||
</title>
|
||
<section><title> Overview</title>
|
||
<para>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:
|
||
<inlinemediaobject>
|
||
<imageobject>
|
||
<imagedata align="middle" valign="middle" fileref="hfilev2.png" />
|
||
</imageobject>
|
||
<textobject>
|
||
<phrase>HFile Version 2</phrase>
|
||
</textobject>
|
||
<caption>
|
||
<para>HFile Version 2
|
||
</para>
|
||
</caption>
|
||
</inlinemediaobject>
|
||
|
||
</para>
|
||
</section>
|
||
<section><title>Unified version 2 block format</title>
|
||
<para>In the version 2 every block in the data section contains the following fields: </para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>8 bytes: Block type, a sequence of bytes equivalent to version 1's "magic records". Supported block types are: </para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>DATA – data blocks
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
LEAF_INDEX – leaf-level index blocks in a multi-level-block-index
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
BLOOM_CHUNK – Bloom filter chunks
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
META – meta blocks (not used for Bloom filters in version 2 anymore)
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
INTERMEDIATE_INDEX – intermediate-level index blocks in a multi-level blockindex
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
ROOT_INDEX – root>level index blocks in a multi>level block index
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
FILE_INFO – the “file info” block, a small key>value map of metadata
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
BLOOM_META – a Bloom filter metadata block in the load>on>open section
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
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
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
INDEX_V1 – this block type is only used for legacy HFile v1 block
|
||
</para>
|
||
</listitem>
|
||
</orderedlist>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Compressed size of the block's data, not including the header (int).
|
||
</para>
|
||
<para>
|
||
Can be used for skipping the current data block when scanning HFile data.
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Uncompressed size of the block's data, not including the header (int)</para>
|
||
<para>
|
||
This is equal to the compressed size if the compression algorithm is NON
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>File offset of the previous block of the same type (long)</para>
|
||
<para>
|
||
Can be used for seeking to the previous data/index block
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Compressed data (or uncompressed data if the compression algorithm is NONE).</para>
|
||
</listitem>
|
||
</orderedlist>
|
||
<para>The above format of blocks is used in the following HFile sections:</para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>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. </para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>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.
|
||
</para>
|
||
</listitem>
|
||
</orderedlist>
|
||
<para>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. </para></section>
|
||
|
||
<section><title> Block index in version 2</title>
|
||
<para>There are three types of block indexes in HFile version 2, stored in two different formats (root and non-root): </para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>Data index — version 2 multi-level block index, consisting of:</para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>
|
||
Version 2 root index, stored in the data block index section of the file
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
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
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
Optionally, version 2 leaf levels, stored in the non%root format inline with data blocks
|
||
</para>
|
||
</listitem>
|
||
</orderedlist>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Meta index — version 2 root index format only, stored in the meta index section of the file</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Bloom index — version 2 root index format only, stored in the “load-on-open” section as part of Bloom filter metadata.</para>
|
||
</listitem>
|
||
</orderedlist></section>
|
||
<section><title>
|
||
Root block index format in version 2</title>
|
||
<para>This format applies to:</para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>Root level of the version 2 data index</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Entire meta and Bloom indexes in version 2, which are always single-level. </para>
|
||
</listitem>
|
||
</orderedlist>
|
||
<para>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. </para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>Offset (long) </para>
|
||
<para>
|
||
This offset may point to a data block or to a deeper>level index block.
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>On-disk size (int) </para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Key (a serialized byte array stored using Bytes.writeByteArray) </para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>Key (VInt)
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Key bytes
|
||
</para>
|
||
</listitem>
|
||
</orderedlist>
|
||
</listitem>
|
||
</orderedlist>
|
||
<para>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.</para>
|
||
|
||
<para>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:</para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>Middle leaf index block offset</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Middle leaf block on-disk size (meaning the leaf index block containing the reference to the “middle” data block of the file) </para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>The index of the mid-key (defined below) in the middle leaf-level block.</para>
|
||
</listitem>
|
||
</orderedlist>
|
||
<para/>
|
||
<para>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. </para>
|
||
<para/>
|
||
<para>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.</para></section>
|
||
<section><title>
|
||
Non-root block index format in version 2</title>
|
||
<para>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. </para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>numEntries: the number of entries (int). </para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>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}.</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>Entries. Each entry contains: </para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>
|
||
Offset of the block referenced by this entry in the file (long)
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
On>disk size of the referenced block (int)
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
Key. The length can be calculated from entryOffsets.
|
||
</para>
|
||
</listitem>
|
||
</orderedlist>
|
||
|
||
</listitem>
|
||
</orderedlist></section><section><title>
|
||
Bloom filters in version 2</title>
|
||
<para>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. </para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>A compound Bloom filter. </para>
|
||
<orderedlist>
|
||
<listitem>
|
||
<para>
|
||
Bloom filter version = 3 (int). There used to be a DynamicByteBloomFilter class that had the Bloom filter version number 2
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
The total byte size of all compound Bloom filter chunks (long)
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
Number of hash functions (int
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
Type of hash functions (int)
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
The total key count inserted into the Bloom filter (long)
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
The maximum total number of keys in the Bloom filter (long)
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
The number of chunks (int)
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
Comparator class used for Bloom filter keys, a UTF>8 encoded string stored using Bytes.writeByteArray
|
||
</para>
|
||
</listitem>
|
||
<listitem>
|
||
<para>
|
||
Bloom block index in the version 2 root block index format
|
||
</para>
|
||
</listitem>
|
||
</orderedlist>
|
||
</listitem>
|
||
</orderedlist></section><section><title>File Info format in versions 1 and 2</title>
|
||
<para>The file info block is a serialized <ulink url="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/io/HbaseMapWritable.html">HbaseMapWritable</ulink> (essentially a map from byte arrays to byte arrays) with the following keys, among others. StoreFile-level logic adds more keys to this.</para>
|
||
<informaltable frame="all">
|
||
<tgroup cols="2"><tbody><row>
|
||
<entry>
|
||
<para>hfile.LASTKEY </para>
|
||
</entry>
|
||
<entry>
|
||
<para>The last key of the file (byte array) </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry>
|
||
<para>hfile.AVG_KEY_LEN </para>
|
||
</entry>
|
||
<entry>
|
||
<para>The average key length in the file (int) </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry>
|
||
<para>hfile.AVG_VALUE_LEN </para>
|
||
</entry>
|
||
<entry>
|
||
<para>The average value length in the file (int) </para>
|
||
</entry>
|
||
</row></tbody></tgroup>
|
||
</informaltable>
|
||
<para>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.</para></section><section><title>
|
||
Fixed file trailer format differences between versions 1 and 2</title>
|
||
<para>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. </para>
|
||
<para/>
|
||
<informaltable frame="all">
|
||
<tgroup cols="2">
|
||
<colspec colname='c1'/>
|
||
<colspec colname='c2'/>
|
||
<tbody>
|
||
<row>
|
||
<entry>
|
||
<para>Version 1 </para>
|
||
</entry>
|
||
<entry>
|
||
<para>Version 2 </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry align="center" namest="c1" nameend="c2">
|
||
<para>File info offset (long) </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry>
|
||
<para>Data index offset (long) </para>
|
||
</entry>
|
||
<entry>
|
||
<para>loadOnOpenOffset (long)</para>
|
||
<para><emphasis>The offset of the section that we need toload when opening the file.</emphasis></para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry align="center" namest="c1" nameend="c2">
|
||
<para>Number of data index entries (int) </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry>
|
||
<para>metaIndexOffset (long)</para>
|
||
<para>This field is not being used by the version 1 reader, so we removed it from version 2.</para>
|
||
</entry>
|
||
<entry>
|
||
<para>uncompressedDataIndexSize (long)</para>
|
||
<para>The total uncompressed size of the whole data block index, including root-level, intermediate-level, and leaf-level blocks.</para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry namest="c1" nameend="c2" align="center">
|
||
<para>Number of meta index entries (int) </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry namest="c1" nameend="c2" align="center">
|
||
<para>Total uncompressed bytes (long) </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry>
|
||
<para>numEntries (int) </para>
|
||
</entry>
|
||
<entry>
|
||
<para>numEntries (long) </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry namest="c1" nameend="c2" align="center">
|
||
<para>Compression codec: 0 = LZO, 1 = GZ, 2 = NONE (int) </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry>
|
||
<para></para>
|
||
</entry>
|
||
<entry>
|
||
<para>The number of levels in the data block index (int) </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry>
|
||
<para></para>
|
||
</entry>
|
||
<entry>
|
||
<para>firstDataBlockOffset (long)</para>
|
||
<para>The offset of the first first data block. Used when scanning. </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry>
|
||
<para></para>
|
||
</entry>
|
||
<entry>
|
||
<para>lastDataBlockEnd (long)</para>
|
||
<para>The offset of the first byte after the last key/value data block. We don't need to go beyond this offset when scanning. </para>
|
||
</entry>
|
||
</row>
|
||
<row>
|
||
<entry>
|
||
<para>Version: 1 (int) </para>
|
||
</entry>
|
||
<entry>
|
||
<para>Version: 2 (int) </para>
|
||
</entry>
|
||
</row></tbody></tgroup>
|
||
</informaltable>
|
||
<para/></section></section></appendix>
|
||
|
||
<appendix xml:id="other.info">
|
||
<title>Other Information About HBase</title>
|
||
<section xml:id="other.info.videos"><title>HBase Videos</title>
|
||
<para>Introduction to HBase
|
||
<itemizedlist>
|
||
<listitem><link xlink:href="http://www.cloudera.com/videos/chicago_data_summit_apache_hbase_an_introduction_todd_lipcon">Introduction to HBase</link> by Todd Lipcon (Chicago Data Summit 2011).
|
||
</listitem>
|
||
<listitem><link xlink:href="http://www.cloudera.com/videos/intorduction-hbase-todd-lipcon">Introduction to HBase</link> by Todd Lipcon (2010).
|
||
</listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
<para><link xlink:href="http://www.cloudera.com/videos/hadoop-world-2011-presentation-video-building-realtime-big-data-services-at-facebook-with-hadoop-and-hbase">Building Real Time Services at Facebook with HBase</link> by Jonathan Gray (Hadoop World 2011).
|
||
</para>
|
||
<para><link xlink:href="http://www.cloudera.com/videos/hw10_video_how_stumbleupon_built_and_advertising_platform_using_hbase_and_hadoop">HBase and Hadoop, Mixing Real-Time and Batch Processing at StumbleUpon</link> by JD Cryans (Hadoop World 2010).
|
||
</para>
|
||
</section>
|
||
<section xml:id="other.info.pres"><title>HBase Presentations (Slides)</title>
|
||
<para><link xlink:href="http://www.cloudera.com/resource/hadoop-world-2011-presentation-slides-advanced-hbase-schema-design">Advanced HBase Schema Design</link> by Lars George (Hadoop World 2011).
|
||
</para>
|
||
<para><link xlink:href="http://www.slideshare.net/cloudera/chicago-data-summit-apache-hbase-an-introduction">Introduction to HBase</link> by Todd Lipcon (Chicago Data Summit 2011).
|
||
</para>
|
||
<para><link xlink:href="http://www.slideshare.net/cloudera/hw09-practical-h-base-getting-the-most-from-your-h-base-install">Getting The Most From Your HBase Install</link> by Ryan Rawson, Jonathan Gray (Hadoop World 2009).
|
||
</para>
|
||
</section>
|
||
<section xml:id="other.info.papers"><title>HBase Papers</title>
|
||
<para><link xlink:href="http://research.google.com/archive/bigtable.html">BigTable</link> by Google (2006).
|
||
</para>
|
||
<para><link xlink:href="http://www.larsgeorge.com/2010/05/hbase-file-locality-in-hdfs.html">HBase and HDFS Locality</link> by Lars George (2010).
|
||
</para>
|
||
<para><link xlink:href="http://ianvarley.com/UT/MR/Varley_MastersReport_Full_2009-08-07.pdf">No Relation: The Mixed Blessings of Non-Relational Databases</link> by Ian Varley (2009).
|
||
</para>
|
||
</section>
|
||
<section xml:id="other.info.sites"><title>HBase Sites</title>
|
||
<para><link xlink:href="http://www.cloudera.com/blog/category/hbase/">Cloudera's HBase Blog</link> has a lot of links to useful HBase information.
|
||
<itemizedlist>
|
||
<listitem><link xlink:href="http://www.cloudera.com/blog/2010/04/cap-confusion-problems-with-partition-tolerance/">CAP Confusion</link> is a relevant entry for background information on
|
||
distributed storage systems.
|
||
</listitem>
|
||
</itemizedlist>
|
||
</para>
|
||
<para><link xlink:href="http://wiki.apache.org/hadoop/HBase/HBasePresentations">HBase Wiki</link> has a page with a number of presentations.
|
||
</para>
|
||
</section>
|
||
<section xml:id="other.info.books"><title>HBase Books</title>
|
||
<para><link xlink:href="http://shop.oreilly.com/product/0636920014348.do">HBase: The Definitive Guide</link> by Lars George.
|
||
</para>
|
||
</section>
|
||
<section xml:id="other.info.books.hadoop"><title>Hadoop Books</title>
|
||
<para><link xlink:href="http://shop.oreilly.com/product/9780596521981.do">Hadoop: The Definitive Guide</link> by Tom White.
|
||
</para>
|
||
</section>
|
||
|
||
</appendix>
|
||
|
||
<appendix xml:id="hbase.history"><title>HBase History</title>
|
||
<itemizedlist>
|
||
<listitem>2006: <link xlink:href="http://research.google.com/archive/bigtable.html">BigTable</link> paper published by Google.
|
||
</listitem>
|
||
<listitem>2006 (end of year): HBase development starts.
|
||
</listitem>
|
||
<listitem>2008: HBase becomes Hadoop sub-project.
|
||
</listitem>
|
||
<listitem>2010: HBase becomes Apache top-level project.
|
||
</listitem>
|
||
</itemizedlist>
|
||
</appendix>
|
||
|
||
<appendix xml:id="asf" ><title>HBase and the Apache Software Foundation</title>
|
||
<para>HBase is a project in the Apache Software Foundation and as such there are responsibilities to the ASF to ensure
|
||
a healthy project.</para>
|
||
<section xml:id="asf.devprocess"><title>ASF Development Process</title>
|
||
<para>See the <link xlink:href="http://www.apache.org/dev/#committers">Apache Development Process page</link>
|
||
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.
|
||
</para>
|
||
</section>
|
||
<section xml:id="asf.reporting"><title>ASF Board Reporting</title>
|
||
<para>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 <link xlink:href="http://www.apache.org/foundation/board/reporting">ASF board reporting</link> for more information.
|
||
</para>
|
||
</section>
|
||
</appendix>
|
||
|
||
<index xml:id="book_index">
|
||
<title>Index</title>
|
||
</index>
|
||
</book>
|