Package org.apache.hadoop.hbase.mapreduce

Provides HBase MapReduce Input/OutputFormats, a table indexing MapReduce job, and utility

See:
          Description

Class Summary
BuildTableIndex Example table column indexing class.
Driver Driver for hbase mapreduce jobs.
Export Export an HBase table.
GroupingTableMapper Extract grouping columns from input record.
HFileOutputFormat Writes HFiles.
HRegionPartitioner<KEY,VALUE> This is used to partition the output keys into groups of keys.
IdentityTableMapper Pass the given key and record as-is to the reduce phase.
IdentityTableReducer Convenience class that simply writes all values (which must be Put or Delete instances) passed to it out to the configured HBase table.
Import Import data written by Export.
IndexConfiguration Configuration parameters for building a Lucene index.
IndexConfiguration.ColumnConf  
IndexOutputFormat Create a local index, unwrap Lucene documents created by reduce, add them to the index, and copy the index to the destination.
IndexRecordWriter Writes the records into a Lucene index writer.
IndexTableReducer Construct a Lucene document per row, which is consumed by IndexOutputFormat to build a Lucene index
KeyValueSortReducer Emits sorted KeyValues.
LuceneDocumentWrapper A utility class used to pass a lucene document from reduce to OutputFormat.
RowCounter A job with a just a map phase to count rows.
TableInputFormat Convert HBase tabular data into a format that is consumable by Map/Reduce.
TableInputFormatBase A base for TableInputFormats.
TableMapper<KEYOUT,VALUEOUT> Extends the base Mapper class to add the required input key and value classes.
TableMapReduceUtil Utility for TableMapper and TableReducer
TableOutputCommitter Small committer class that does not do anything.
TableOutputFormat<KEY> Convert Map/Reduce output and write it to an HBase table.
TableOutputFormat.TableRecordWriter<KEY> Writes the reducer output to an HBase table.
TableReducer<KEYIN,VALUEIN,KEYOUT> Extends the basic Reducer class to add the required key and value input/output classes.
TableSplit A table split corresponds to a key range (low, high).
 

Package org.apache.hadoop.hbase.mapreduce Description

Provides HBase MapReduce Input/OutputFormats, a table indexing MapReduce job, and utility

Table of Contents

HBase, MapReduce and the CLASSPATH

MapReduce jobs deployed to a MapReduce cluster do not by default have access to the HBase configuration under $HBASE_CONF_DIR nor to HBase classes. You could add hbase-site.xml to $HADOOP_HOME/conf and add hbase jars to the $HADOOP_HOME/lib and copy these changes across your cluster but a cleaner means of adding hbase configuration and classes to the cluster CLASSPATH is by uncommenting HADOOP_CLASSPATH in $HADOOP_HOME/conf/hadoop-env.sh adding hbase dependencies here. For example, here is how you would amend hadoop-env.sh adding the built hbase jar, zookeeper (needed by hbase client), hbase conf, and the PerformanceEvaluation class from the built hbase test jar to the hadoop CLASSPATH:

# Extra Java CLASSPATH elements. Optional.
# export HADOOP_CLASSPATH=
export HADOOP_CLASSPATH=$HBASE_HOME/build/hbase-X.X.X.jar:$HBASE_HOME/build/hbase-X.X.X-test.jar:$HBASE_HOME/conf:${HBASE_HOME}/lib/zookeeper-X.X.X.jar

Expand $HBASE_HOME in the above appropriately to suit your local environment.

After copying the above change around your cluster (and restarting), this is how you would run the PerformanceEvaluation MR job to put up 4 clients (Presumes a ready mapreduce cluster):

$HADOOP_HOME/bin/hadoop org.apache.hadoop.hbase.PerformanceEvaluation sequentialWrite 4

Another possibility, if for example you do not have access to hadoop-env.sh or are unable to restart the hadoop cluster, is bundling the hbase jars into a mapreduce job jar adding it and its dependencies under the job jar lib/ directory and the hbase conf into the job jars top-level directory.

HBase as MapReduce job data source and sink

HBase can be used as a data source, TableInputFormat, and data sink, TableOutputFormat, for MapReduce jobs. Writing MapReduce jobs that read or write HBase, you'll probably want to subclass TableMapper and/or TableReducer. See the do-nothing pass-through classes IdentityTableMapper and IdentityTableReducer for basic usage. For a more involved example, see RowCounter or review the org.apache.hadoop.hbase.mapreduce.TestTableMapReduce unit test.

Running mapreduce jobs that have hbase as source or sink, you'll need to specify source/sink table and column names in your configuration.

Reading from hbase, the TableInputFormat asks hbase for the list of regions and makes a map-per-region or mapred.map.tasks maps, whichever is smaller (If your job only has two maps, up mapred.map.tasks to a number > number of regions). Maps will run on the adjacent TaskTracker if you are running a TaskTracer and RegionServer per node. Writing, it may make sense to avoid the reduce step and write yourself back into hbase from inside your map. You'd do this when your job does not need the sort and collation that mapreduce does on the map emitted data; on insert, hbase 'sorts' so there is no point double-sorting (and shuffling data around your mapreduce cluster) unless you need to. If you do not need the reduce, you might just have your map emit counts of records processed just so the framework's report at the end of your job has meaning or set the number of reduces to zero and use TableOutputFormat. See example code below. If running the reduce step makes sense in your case, its usually better to have lots of reducers so load is spread across the hbase cluster.

There is also a new hbase partitioner that will run as many reducers as currently existing regions. The HRegionPartitioner is suitable when your table is large and your upload is not such that it will greatly alter the number of existing regions when done; otherwise use the default partitioner.

Bulk import writing HFiles directly

If importing into a new table, its possible to by-pass the HBase API and write your content directly to the filesystem properly formatted as HBase data files (HFiles). Your import will run faster, perhaps as much as an order of magnitude faster if not more.

You will need to write a MapReduce job. The map task will know how to pull from your data source. Your reduce task will need to be hooked up to HFileOutputFormat. It expects to receive a row id and a value. The row id must be formatted as a ImmutableBytesWritable and the value as a KeyValue (A KeyValue holds he value for a cell and its coordinates; row/family/qualifier/timestamp, etc.). Your reduce task will also need to emit the KeyValues in order. See KeyValueSortReducer for an example reducer that emits KeyValues in order.

Most importantly, you will also need to ensure that your MapReduce job ensures a total ordering among all keys. MapReduce by default distributes keys among reducers using a Partitioner that hashes on the map task output key: i.e. the reducer a key ends up in is by default determined as follows (key.hashCode() & Integer.MAX_VALUE) % numReduceTasks. Keys are sorted by the MapReduce framework before they are passed to the reducer BUT the sort is scoped to the particular reducer. Its not a global sort. Given the default hash Partitioner, if the keys were 0-4 (inclusive), and you had configured two reducers, reducer 0 would have get keys 0, 2 and 4 whereas reducer 1 would get keys 1 and 3 (in order). For your bulk import to work, the keys need to be orderd so reducer 0 gets keys 0-2 and reducer 1 gets keys 3-4 (See TotalOrderPartitioner up in hadoop for more on what this means). To achieve total ordering, you will likely need to write a Partitioner that is intimate with your tables key namespace and that knows how to distribute keys among the reducers so a total order is maintained.

See org.apache.hadoop.hbase.mapreduce.TestHFileOutputFormat for an example that puts together KeyValueSortReducer and HFileOutputFormat.

HFileOutputFormat writes HFiles. When your MapReduce file finishes, in your output directory you will have many HFiles. Run the script bin/loadtable.rb to move the files from the MapReduce output directory under hbase. See head of script for how to run it. This script also adds the new table data to the hbase catalog tables. When the script completes, on the next run of the hbase metascanner -- it usually runs every minute -- your new table should be visible and populated.

Example Code

Sample Row Counter

See RowCounter. This job uses TableInputFormat and does a count of all rows in specified table. You should be able to run it by doing: % ./bin/hadoop jar hbase-X.X.X.jar. This will invoke the hbase MapReduce Driver class. Select 'rowcounter' from the choice of jobs offered. This will emit rowcouner 'usage'. Specify tablename, column to count and output directory. You may need to add the hbase conf directory to $HADOOP_HOME/conf/hadoop-env.sh#HADOOP_CLASSPATH so the rowcounter gets pointed at the right hbase cluster (or, build a new jar with an appropriate hbase-site.xml built into your job jar).



Copyright © 2009 The Apache Software Foundation