The following is an example of using HBase as a MapReduce source in read-only manner. Specifically, there is a Mapper instance but no Reducer, and nothing is being emitted from the Mapper. There job would be defined as follows...
Configuration config = HBaseConfiguration.create(); Job job = new Job(config, "ExampleRead"); job.setJarByClass(MyReadJob.class); // class that contains mapper Scan scan = new Scan(); scan.setCaching(500); // 1 is the default in Scan, which will be bad for MapReduce jobs scan.setCacheBlocks(false); // don't set to true for MR jobs // set other scan attrs ... TableMapReduceUtil.initTableMapperJob( tableName, // input HBase table name scan, // Scan instance to control CF and attribute selection MyMapper.class, // mapper null, // mapper output key null, // mapper output value job); job.setOutputFormatClass(NullOutputFormat.class); // because we aren't emitting anything from mapper boolean b = job.waitForCompletion(true); if (!b) { throw new IOException("error with job!"); }
...and the mapper instance would extend TableMapper...
public static class MyMapper extends TableMapper<Text, Text> { public void map(ImmutableBytesWritable row, Result value, Context context) throws InterruptedException, IOException { // process data for the row from the Result instance. } }
The following is an example of using HBase both as a source and as a sink with MapReduce. This example will simply copy data from one table to another.
Configuration config = HBaseConfiguration.create(); Job job = new Job(config,"ExampleReadWrite"); job.setJarByClass(MyReadWriteJob.class); // class that contains mapper Scan scan = new Scan(); scan.setCaching(500); // 1 is the default in Scan, which will be bad for MapReduce jobs scan.setCacheBlocks(false); // don't set to true for MR jobs // set other scan attrs TableMapReduceUtil.initTableMapperJob( sourceTable, // input table scan, // Scan instance to control CF and attribute selection MyMapper.class, // mapper class null, // mapper output key null, // mapper output value job); TableMapReduceUtil.initTableReducerJob( targetTable, // output table null, // reducer class job); job.setNumReduceTasks(0); boolean b = job.waitForCompletion(true); if (!b) { throw new IOException("error with job!"); }
An explanation is required of what TableMapReduceUtil
is doing,
especially with the reducer. TableOutputFormat
is being used as the outputFormat class, and several parameters are being set on the
config (e.g., TableOutputFormat.OUTPUT_TABLE), as well as setting the reducer output key
to ImmutableBytesWritable
and reducer value to
Writable
. These could be set by the programmer on the job and
conf, but TableMapReduceUtil
tries to make things easier.
The following is the example mapper, which will create a Put
and matching the input Result
and emit it. Note: this is what the
CopyTable utility does.
public static class MyMapper extends TableMapper<ImmutableBytesWritable, Put> { public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException, InterruptedException { // this example is just copying the data from the source table... context.write(row, resultToPut(row,value)); } private static Put resultToPut(ImmutableBytesWritable key, Result result) throws IOException { Put put = new Put(key.get()); for (KeyValue kv : result.raw()) { put.add(kv); } return put; } }
There isn't actually a reducer step, so TableOutputFormat
takes
care of sending the Put
to the target table.
This is just an example, developers could choose not to use
TableOutputFormat
and connect to the target table themselves.
TODO: example for MultiTableOutputFormat
.
The following example uses HBase as a MapReduce source and sink with a summarization step. This example will count the number of distinct instances of a value in a table and write those summarized counts in another table.
Configuration config = HBaseConfiguration.create(); Job job = new Job(config,"ExampleSummary"); job.setJarByClass(MySummaryJob.class); // class that contains mapper and reducer Scan scan = new Scan(); scan.setCaching(500); // 1 is the default in Scan, which will be bad for MapReduce jobs scan.setCacheBlocks(false); // don't set to true for MR jobs // set other scan attrs TableMapReduceUtil.initTableMapperJob( sourceTable, // input table scan, // Scan instance to control CF and attribute selection MyMapper.class, // mapper class Text.class, // mapper output key IntWritable.class, // mapper output value job); TableMapReduceUtil.initTableReducerJob( targetTable, // output table MyTableReducer.class, // reducer class job); job.setNumReduceTasks(1); // at least one, adjust as required boolean b = job.waitForCompletion(true); if (!b) { throw new IOException("error with job!"); }
In this example mapper a column with a String-value is chosen as the value to summarize
upon. This value is used as the key to emit from the mapper, and an
IntWritable
represents an instance counter.
public static class MyMapper extends TableMapper<Text, IntWritable> { public static final byte[] CF = "cf".getBytes(); public static final byte[] ATTR1 = "attr1".getBytes(); 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(CF, ATTR1)); text.set(val); // we can only emit Writables... context.write(text, ONE); } }
In the reducer, the "ones" are counted (just like any other MR example that does this),
and then emits a Put
.
public static class MyTableReducer extends TableReducer<Text, IntWritable, ImmutableBytesWritable> { public static final byte[] CF = "cf".getBytes(); public static final byte[] COUNT = "count".getBytes(); 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(CF, COUNT, Bytes.toBytes(i)); context.write(null, put); } }
This very similar to the summary example above, with exception that this is using HBase as a MapReduce source but HDFS as the sink. The differences are in the job setup and in the reducer. The mapper remains the same.
Configuration config = HBaseConfiguration.create(); Job job = new Job(config,"ExampleSummaryToFile"); job.setJarByClass(MySummaryFileJob.class); // class that contains mapper and reducer Scan scan = new Scan(); scan.setCaching(500); // 1 is the default in Scan, which will be bad for MapReduce jobs scan.setCacheBlocks(false); // don't set to true for MR jobs // set other scan attrs TableMapReduceUtil.initTableMapperJob( sourceTable, // input table scan, // Scan instance to control CF and attribute selection MyMapper.class, // mapper class Text.class, // mapper output key IntWritable.class, // mapper output value job); job.setReducerClass(MyReducer.class); // reducer class job.setNumReduceTasks(1); // at least one, adjust as required FileOutputFormat.setOutputPath(job, new Path("/tmp/mr/mySummaryFile")); // adjust directories as required boolean b = job.waitForCompletion(true); if (!b) { throw new IOException("error with job!"); }
As stated above, the previous Mapper can run unchanged with this example. As for the Reducer, it is a "generic" Reducer instead of extending TableMapper and emitting Puts.
public static class MyReducer extends Reducer<Text, IntWritable, Text, IntWritable> { public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException { int i = 0; for (IntWritable val : values) { i += val.get(); } context.write(key, new IntWritable(i)); } }
It is also possible to perform summaries without a reducer - if you use HBase as the reducer.
An HBase target table would need to exist for the job summary. The HTable method
incrementColumnValue
would be used to atomically increment values. From a
performance perspective, it might make sense to keep a Map of values with their values to
be incremeneted for each map-task, and make one update per key at during the
cleanup
method of the mapper. However, your milage may vary depending on the
number of rows to be processed and unique keys.
In the end, the summary results are in HBase.
Sometimes it is more appropriate to generate summaries to an RDBMS. For these cases,
it is possible to generate summaries directly to an RDBMS via a custom reducer. The
setup
method can connect to an RDBMS (the connection information can be
passed via custom parameters in the context) and the cleanup method can close the
connection.
It is critical to understand that number of reducers for the job affects the summarization implementation, and you'll have to design this into your reducer. Specifically, whether it is designed to run as a singleton (one reducer) or multiple reducers. Neither is right or wrong, it depends on your use-case. Recognize that the more reducers that are assigned to the job, the more simultaneous connections to the RDBMS will be created - this will scale, but only to a point.
public static class MyRdbmsReducer extends Reducer<Text, IntWritable, Text, IntWritable> { private Connection c = null; public void setup(Context context) { // create DB connection... } public void reduce(Text key, Iterable<IntWritable> values, Context context) throws IOException, InterruptedException { // do summarization // in this example the keys are Text, but this is just an example } public void cleanup(Context context) { // close db connection } }
In the end, the summary results are written to your RDBMS table/s.