[120] | 1 | /** |
---|
| 2 | * Licensed to the Apache Software Foundation (ASF) under one |
---|
| 3 | * or more contributor license agreements. See the NOTICE file |
---|
| 4 | * distributed with this work for additional information |
---|
| 5 | * regarding copyright ownership. The ASF licenses this file |
---|
| 6 | * to you under the Apache License, Version 2.0 (the |
---|
| 7 | * "License"); you may not use this file except in compliance |
---|
| 8 | * with the License. You may obtain a copy of the License at |
---|
| 9 | * |
---|
| 10 | * http://www.apache.org/licenses/LICENSE-2.0 |
---|
| 11 | * |
---|
| 12 | * Unless required by applicable law or agreed to in writing, software |
---|
| 13 | * distributed under the License is distributed on an "AS IS" BASIS, |
---|
| 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
---|
| 15 | * See the License for the specific language governing permissions and |
---|
| 16 | * limitations under the License. |
---|
| 17 | */ |
---|
| 18 | |
---|
| 19 | package org.apache.hadoop.mapreduce; |
---|
| 20 | |
---|
| 21 | import java.io.IOException; |
---|
| 22 | |
---|
| 23 | import org.apache.hadoop.conf.Configuration; |
---|
| 24 | import org.apache.hadoop.io.RawComparator; |
---|
| 25 | import org.apache.hadoop.io.compress.CompressionCodec; |
---|
| 26 | |
---|
| 27 | /** |
---|
| 28 | * Maps input key/value pairs to a set of intermediate key/value pairs. |
---|
| 29 | * |
---|
| 30 | * <p>Maps are the individual tasks which transform input records into a |
---|
| 31 | * intermediate records. The transformed intermediate records need not be of |
---|
| 32 | * the same type as the input records. A given input pair may map to zero or |
---|
| 33 | * many output pairs.</p> |
---|
| 34 | * |
---|
| 35 | * <p>The Hadoop Map-Reduce framework spawns one map task for each |
---|
| 36 | * {@link InputSplit} generated by the {@link InputFormat} for the job. |
---|
| 37 | * <code>Mapper</code> implementations can access the {@link Configuration} for |
---|
| 38 | * the job via the {@link JobContext#getConfiguration()}. |
---|
| 39 | * |
---|
| 40 | * <p>The framework first calls |
---|
| 41 | * {@link #setup(org.apache.hadoop.mapreduce.Mapper.Context)}, followed by |
---|
| 42 | * {@link #map(Object, Object, Context)} |
---|
| 43 | * for each key/value pair in the <code>InputSplit</code>. Finally |
---|
| 44 | * {@link #cleanup(Context)} is called.</p> |
---|
| 45 | * |
---|
| 46 | * <p>All intermediate values associated with a given output key are |
---|
| 47 | * subsequently grouped by the framework, and passed to a {@link Reducer} to |
---|
| 48 | * determine the final output. Users can control the sorting and grouping by |
---|
| 49 | * specifying two key {@link RawComparator} classes.</p> |
---|
| 50 | * |
---|
| 51 | * <p>The <code>Mapper</code> outputs are partitioned per |
---|
| 52 | * <code>Reducer</code>. Users can control which keys (and hence records) go to |
---|
| 53 | * which <code>Reducer</code> by implementing a custom {@link Partitioner}. |
---|
| 54 | * |
---|
| 55 | * <p>Users can optionally specify a <code>combiner</code>, via |
---|
| 56 | * {@link Job#setCombinerClass(Class)}, to perform local aggregation of the |
---|
| 57 | * intermediate outputs, which helps to cut down the amount of data transferred |
---|
| 58 | * from the <code>Mapper</code> to the <code>Reducer</code>. |
---|
| 59 | * |
---|
| 60 | * <p>Applications can specify if and how the intermediate |
---|
| 61 | * outputs are to be compressed and which {@link CompressionCodec}s are to be |
---|
| 62 | * used via the <code>Configuration</code>.</p> |
---|
| 63 | * |
---|
| 64 | * <p>If the job has zero |
---|
| 65 | * reduces then the output of the <code>Mapper</code> is directly written |
---|
| 66 | * to the {@link OutputFormat} without sorting by keys.</p> |
---|
| 67 | * |
---|
| 68 | * <p>Example:</p> |
---|
| 69 | * <p><blockquote><pre> |
---|
| 70 | * public class TokenCounterMapper |
---|
| 71 | * extends Mapper<Object, Text, Text, IntWritable>{ |
---|
| 72 | * |
---|
| 73 | * private final static IntWritable one = new IntWritable(1); |
---|
| 74 | * private Text word = new Text(); |
---|
| 75 | * |
---|
| 76 | * public void map(Object key, Text value, Context context) throws IOException { |
---|
| 77 | * StringTokenizer itr = new StringTokenizer(value.toString()); |
---|
| 78 | * while (itr.hasMoreTokens()) { |
---|
| 79 | * word.set(itr.nextToken()); |
---|
| 80 | * context.collect(word, one); |
---|
| 81 | * } |
---|
| 82 | * } |
---|
| 83 | * } |
---|
| 84 | * </pre></blockquote></p> |
---|
| 85 | * |
---|
| 86 | * <p>Applications may override the {@link #run(Context)} method to exert |
---|
| 87 | * greater control on map processing e.g. multi-threaded <code>Mapper</code>s |
---|
| 88 | * etc.</p> |
---|
| 89 | * |
---|
| 90 | * @see InputFormat |
---|
| 91 | * @see JobContext |
---|
| 92 | * @see Partitioner |
---|
| 93 | * @see Reducer |
---|
| 94 | */ |
---|
| 95 | public class Mapper<KEYIN, VALUEIN, KEYOUT, VALUEOUT> { |
---|
| 96 | |
---|
| 97 | public class Context |
---|
| 98 | extends MapContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT> { |
---|
| 99 | public Context(Configuration conf, TaskAttemptID taskid, |
---|
| 100 | RecordReader<KEYIN,VALUEIN> reader, |
---|
| 101 | RecordWriter<KEYOUT,VALUEOUT> writer, |
---|
| 102 | OutputCommitter committer, |
---|
| 103 | StatusReporter reporter, |
---|
| 104 | InputSplit split) throws IOException, InterruptedException { |
---|
| 105 | super(conf, taskid, reader, writer, committer, reporter, split); |
---|
| 106 | } |
---|
| 107 | } |
---|
| 108 | |
---|
| 109 | /** |
---|
| 110 | * Called once at the beginning of the task. |
---|
| 111 | */ |
---|
| 112 | protected void setup(Context context |
---|
| 113 | ) throws IOException, InterruptedException { |
---|
| 114 | // NOTHING |
---|
| 115 | } |
---|
| 116 | |
---|
| 117 | /** |
---|
| 118 | * Called once for each key/value pair in the input split. Most applications |
---|
| 119 | * should override this, but the default is the identity function. |
---|
| 120 | */ |
---|
| 121 | @SuppressWarnings("unchecked") |
---|
| 122 | protected void map(KEYIN key, VALUEIN value, |
---|
| 123 | Context context) throws IOException, InterruptedException { |
---|
| 124 | context.write((KEYOUT) key, (VALUEOUT) value); |
---|
| 125 | } |
---|
| 126 | |
---|
| 127 | /** |
---|
| 128 | * Called once at the end of the task. |
---|
| 129 | */ |
---|
| 130 | protected void cleanup(Context context |
---|
| 131 | ) throws IOException, InterruptedException { |
---|
| 132 | // NOTHING |
---|
| 133 | } |
---|
| 134 | |
---|
| 135 | /** |
---|
| 136 | * Expert users can override this method for more complete control over the |
---|
| 137 | * execution of the Mapper. |
---|
| 138 | * @param context |
---|
| 139 | * @throws IOException |
---|
| 140 | */ |
---|
| 141 | public void run(Context context) throws IOException, InterruptedException { |
---|
| 142 | setup(context); |
---|
| 143 | while (context.nextKeyValue()) { |
---|
| 144 | map(context.getCurrentKey(), context.getCurrentValue(), context); |
---|
| 145 | } |
---|
| 146 | cleanup(context); |
---|
| 147 | } |
---|
| 148 | } |
---|