@InterfaceAudience.Public @InterfaceStability.Stable public class Mapper<KEYIN,VALUEIN,KEYOUT,VALUEOUT> extends Object
Maps are the individual tasks which transform input records into a intermediate records. The transformed intermediate records need not be of the same type as the input records. A given input pair may map to zero or many output pairs.
The Hadoop Map-Reduce framework spawns one map task for each
InputSplit
generated by the InputFormat
for the job.
Mapper
implementations can access the Configuration
for
the job via the JobContext.getConfiguration()
.
The framework first calls
setup(org.apache.hadoop.mapreduce.Mapper.Context)
, followed by
map(Object, Object, org.apache.hadoop.mapreduce.Mapper.Context)
for each key/value pair in the InputSplit
. Finally
cleanup(org.apache.hadoop.mapreduce.Mapper.Context)
is called.
All intermediate values associated with a given output key are
subsequently grouped by the framework, and passed to a Reducer
to
determine the final output. Users can control the sorting and grouping by
specifying two key RawComparator
classes.
The Mapper
outputs are partitioned per
Reducer
. Users can control which keys (and hence records) go to
which Reducer
by implementing a custom Partitioner
.
Users can optionally specify a combiner
, via
Job.setCombinerClass(Class)
, to perform local aggregation of the
intermediate outputs, which helps to cut down the amount of data transferred
from the Mapper
to the Reducer
.
Applications can specify if and how the intermediate
outputs are to be compressed and which CompressionCodec
s are to be
used via the Configuration
.
If the job has zero
reduces then the output of the Mapper
is directly written
to the OutputFormat
without sorting by keys.
Example:
public class TokenCounterMapper extends Mapper<Object, Text, Text, IntWritable>{ private final static IntWritable one = new IntWritable(1); private Text word = new Text(); public void map(Object key, Text value, Context context) throws IOException, InterruptedException { StringTokenizer itr = new StringTokenizer(value.toString()); while (itr.hasMoreTokens()) { word.set(itr.nextToken()); context.write(word, one); } } }
Applications may override the
run(org.apache.hadoop.mapreduce.Mapper.Context)
method to exert
greater control on map processing e.g. multi-threaded Mapper
s
etc.
InputFormat
,
JobContext
,
Partitioner
,
Reducer
Constructor and Description |
---|
Mapper() |
Modifier and Type | Method and Description |
---|---|
protected void |
cleanup(org.apache.hadoop.mapreduce.Mapper.Context context)
Called once at the end of the task.
|
protected void |
map(KEYIN key,
VALUEIN value,
org.apache.hadoop.mapreduce.Mapper.Context context)
Called once for each key/value pair in the input split.
|
void |
run(org.apache.hadoop.mapreduce.Mapper.Context context)
Expert users can override this method for more complete control over the
execution of the Mapper.
|
protected void |
setup(org.apache.hadoop.mapreduce.Mapper.Context context)
Called once at the beginning of the task.
|
public Mapper()
protected void setup(org.apache.hadoop.mapreduce.Mapper.Context context) throws IOException, InterruptedException
IOException
InterruptedException
protected void map(KEYIN key, VALUEIN value, org.apache.hadoop.mapreduce.Mapper.Context context) throws IOException, InterruptedException
IOException
InterruptedException
protected void cleanup(org.apache.hadoop.mapreduce.Mapper.Context context) throws IOException, InterruptedException
IOException
InterruptedException
public void run(org.apache.hadoop.mapreduce.Mapper.Context context) throws IOException, InterruptedException
context
- IOException
InterruptedException
Copyright © 2018 Apache Software Foundation. All rights reserved.