Class ChainReducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
For each record output by the Reducer, the Mapper classes are invoked in a chained (or piped) fashion. The output of the reducer becomes the input of the first mapper and output of first becomes the input of the second, and so on until the last Mapper, the output of the last Mapper will be written to the task's output.
The key functionality of this feature is that the Mappers in the chain do not need to be aware that they are executed after the Reducer or in a chain. This enables having reusable specialized Mappers that can be combined to perform composite operations within a single task.
Special care has to be taken when creating chains that the key/values output by a Mapper are valid for the following Mapper in the chain. It is assumed all Mappers and the Reduce in the chain use matching output and input key and value classes as no conversion is done by the chaining code.
Using the ChainMapper and the ChainReducer classes is possible to
compose Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And
immediate benefit of this pattern is a dramatic reduction in disk IO.
IMPORTANT: There is no need to specify the output key/value classes for the ChainReducer, this is done by the setReducer or the addMapper for the last element in the chain.
ChainReducer usage pattern:
... Job = new Job(conf); .... Configuration reduceConf = new Configuration(false); ... ChainReducer.setReducer(job, XReduce.class, LongWritable.class, Text.class, Text.class, Text.class, true, reduceConf); ChainReducer.addMapper(job, CMap.class, Text.class, Text.class, LongWritable.class, Text.class, false, null); ChainReducer.addMapper(job, DMap.class, LongWritable.class, Text.class, LongWritable.class, LongWritable.class, true, null); ... job.waitForCompletion(true); ...
-
Nested Class Summary
Nested classes/interfaces inherited from class org.apache.hadoop.mapreduce.Reducer
org.apache.hadoop.mapreduce.Reducer.Context -
Constructor Summary
Constructors -
Method Summary
Modifier and TypeMethodDescriptionstatic voidaddMapper(Job job, Class<? extends Mapper> klass, Class<?> inputKeyClass, Class<?> inputValueClass, Class<?> outputKeyClass, Class<?> outputValueClass, Configuration mapperConf) Adds aMapperclass to the chain reducer.voidAdvanced application writers can use theReducer.run(org.apache.hadoop.mapreduce.Reducer.Context)method to control how the reduce task works.static voidsetReducer(Job job, Class<? extends Reducer> klass, Class<?> inputKeyClass, Class<?> inputValueClass, Class<?> outputKeyClass, Class<?> outputValueClass, Configuration reducerConf) Sets theReducerclass to the chain job.protected voidCalled once at the start of the task.
-
Constructor Details
-
ChainReducer
public ChainReducer()
-
-
Method Details
-
setReducer
public static void setReducer(Job job, Class<? extends Reducer> klass, Class<?> inputKeyClass, Class<?> inputValueClass, Class<?> outputKeyClass, Class<?> outputValueClass, Configuration reducerConf) Sets theReducerclass to the chain job.The key and values are passed from one element of the chain to the next, by value. For the added Reducer the configuration given for it,
reducerConf, have precedence over the job's Configuration. This precedence is in effect when the task is running.IMPORTANT: There is no need to specify the output key/value classes for the ChainReducer, this is done by the setReducer or the addMapper for the last element in the chain.
- Parameters:
job- the jobklass- the Reducer class to add.inputKeyClass- reducer input key class.inputValueClass- reducer input value class.outputKeyClass- reducer output key class.outputValueClass- reducer output value class.reducerConf- a configuration for the Reducer class. It is recommended to use a Configuration without default values using theConfiguration(boolean loadDefaults)constructor with FALSE.
-
addMapper
public static void addMapper(Job job, Class<? extends Mapper> klass, Class<?> inputKeyClass, Class<?> inputValueClass, Class<?> outputKeyClass, Class<?> outputValueClass, Configuration mapperConf) throws IOException Adds aMapperclass to the chain reducer.The key and values are passed from one element of the chain to the next, by value For the added Mapper the configuration given for it,
mapperConf, have precedence over the job's Configuration. This precedence is in effect when the task is running.IMPORTANT: There is no need to specify the output key/value classes for the ChainMapper, this is done by the addMapper for the last mapper in the chain.
- Parameters:
job- The job.klass- the Mapper class to add.inputKeyClass- mapper input key class.inputValueClass- mapper input value class.outputKeyClass- mapper output key class.outputValueClass- mapper output value class.mapperConf- a configuration for the Mapper class. It is recommended to use a Configuration without default values using theConfiguration(boolean loadDefaults)constructor with FALSE.- Throws:
IOException
-
setup
protected void setup(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.org.apache.hadoop.mapreduce.Reducer.Context context) Description copied from class:ReducerCalled once at the start of the task. -
run
public void run(Reducer<KEYIN, VALUEIN, throws IOException, InterruptedExceptionKEYOUT, VALUEOUT>.org.apache.hadoop.mapreduce.Reducer.Context context) Description copied from class:ReducerAdvanced application writers can use theReducer.run(org.apache.hadoop.mapreduce.Reducer.Context)method to control how the reduce task works.- Overrides:
runin classReducer<KEYIN,VALUEIN, KEYOUT, VALUEOUT> - Throws:
IOExceptionInterruptedException
-