@Checkpointable @InterfaceAudience.Public @InterfaceStability.Stable public class Reducer<KEYIN,VALUEIN,KEYOUT,VALUEOUT> extends Object
Reducer implementations 
 can access the Configuration for the job via the 
 JobContext.getConfiguration() method.
Reducer has 3 primary phases:
The Reducer copies the sorted output from each 
   Mapper using HTTP across the network.
The framework merge sorts Reducer inputs by 
   keys 
   (since different Mappers may have output the same key).
The shuffle and sort phases occur simultaneously i.e. while outputs are being fetched they are merged.
To achieve a secondary sort on the values returned by the value 
   iterator, the application should extend the key with the secondary
   key and define a grouping comparator. The keys will be sorted using the
   entire key, but will be grouped using the grouping comparator to decide
   which keys and values are sent in the same call to reduce.The grouping 
   comparator is specified via 
   Job.setGroupingComparatorClass(Class). The sort order is
   controlled by 
   Job.setSortComparatorClass(Class).
In this phase the 
   reduce(Object, Iterable, Context)
   method is called for each <key, (collection of values)> in
   the sorted inputs.
The output of the reduce task is typically written to a 
   RecordWriter via 
   TaskInputOutputContext.write(Object, Object).
The output of the Reducer is not re-sorted.
Example:
 public class IntSumReducer<Key> extends Reducer<Key,IntWritable,
                                                 Key,IntWritable> {
   private IntWritable result = new IntWritable();
 
   public void reduce(Key key, Iterable<IntWritable> values,
                      Context context) throws IOException, InterruptedException {
     int sum = 0;
     for (IntWritable val : values) {
       sum += val.get();
     }
     result.set(sum);
     context.write(key, result);
   }
 }
 Mapper, 
Partitioner| Constructor and Description | 
|---|
| Reducer() | 
| Modifier and Type | Method and Description | 
|---|---|
| protected void | cleanup(org.apache.hadoop.mapreduce.Reducer.Context context)Called once at the end of the task. | 
| protected void | reduce(KEYIN key,
            Iterable<VALUEIN> values,
            org.apache.hadoop.mapreduce.Reducer.Context context)This method is called once for each key. | 
| void | run(org.apache.hadoop.mapreduce.Reducer.Context context)Advanced application writers can use the 
  run(org.apache.hadoop.mapreduce.Reducer.Context)method to
 control how the reduce task works. | 
| protected void | setup(org.apache.hadoop.mapreduce.Reducer.Context context)Called once at the start of the task. | 
public Reducer()
protected void setup(org.apache.hadoop.mapreduce.Reducer.Context context) throws IOException, InterruptedException
IOExceptionInterruptedExceptionprotected void reduce(KEYIN key, Iterable<VALUEIN> values, org.apache.hadoop.mapreduce.Reducer.Context context) throws IOException, InterruptedException
IOExceptionInterruptedExceptionprotected void cleanup(org.apache.hadoop.mapreduce.Reducer.Context context) throws IOException, InterruptedException
IOExceptionInterruptedExceptionpublic void run(org.apache.hadoop.mapreduce.Reducer.Context context) throws IOException, InterruptedException
run(org.apache.hadoop.mapreduce.Reducer.Context) method to
 control how the reduce task works.IOExceptionInterruptedExceptionCopyright © 2015 Apache Software Foundation. All Rights Reserved.