HBASE-3618 Add to HBase book, 'schema' chapter - pre-creating regions and key types

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1081435 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-03-14 16:10:12 +00:00
parent cfedd4d7dd
commit 4f5e8d29db
2 changed files with 53 additions and 6 deletions

View File

@ -106,6 +106,8 @@ Release 0.91.0 - Unreleased
HBASE-3631 CLONE - HBase 2984 breaks ability to specify BLOOMFILTER &
COMPRESSION via shell
HBASE-3630 DemoClient.Java is outdated (Moaz Reyed via Stack)
HBASE-3618 Add to HBase book, 'schema' chapter - pre-creating regions and
key types (Doug Meil via Stack)
TASK
HBASE-3559 Move report of split to master OFF the heartbeat channel

View File

@ -1376,13 +1376,18 @@ of all regions.
<title>
Monotonically Increasing Row Keys/Timeseries Data
</title>
<para>See this comic by IKai Lan on why monotically 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>.</para>
<para>If you need to upload time series data into HBase, you should
<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 monotically 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 monoticially 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 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 schema it uses in
HBase. You might also consider just using OpenTSDB altogether.</para>
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 <b>lead</b> 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>
@ -1403,6 +1408,46 @@ of all regions.
names.
`</para>
</section>
<section>
<title>
Table Creation: Pre-Creating Regions
</title>
<para>
Tables in HBase are initially created with one region by default. For bulk imports, this means that all clients will write to the same region until it is large enough to split and become distributed across the cluster. A useful pattern to speed up the bulk import process is to pre-create empty regions. Be somewhat conservative in this, because too-many regions can actually degrade performance. An example of pre-creation using hex-keys is as follows (note: this example may need to be tweaked to the individual applications keys):
</para>
<para>
<pre>
public static boolean createTable(HBaseAdmin admin, HTableDescriptor table, byte[][] splits)
throws IOException {
try {
admin.createTable( table, splits );
return true;
} catch (TableExistsException e) {
logger.info("table " + table.getNameAsString() + " already exists");
// the table already exists...
return false;
}
}
public static byte[][] getHexSplits(String startKey, String endKey, int numRegions) {
byte[][] splits = new byte[numRegions-1][];
BigInteger lowestKey = new BigInteger(startKey, 16);
BigInteger highestKey = new BigInteger(endKey, 16);
BigInteger range = highestKey.subtract(lowestKey);
BigInteger regionIncrement = range.divide(BigInteger.valueOf(numRegions));
lowestKey = lowestKey.add(regionIncrement);
for(int i=0; i &lt; numRegions-1;i++) {
BigInteger key = lowestKey.add(regionIncrement.multiply(BigInteger.valueOf(i)));
byte[] b = String.format("%016x", key).getBytes();
splits[i] = b;
}
return splits;
}
</pre>
</para>
</section>
</chapter>
<chapter xml:id="hbase_metrics">