001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.mapred.lib;
019
020import org.apache.hadoop.classification.InterfaceAudience;
021import org.apache.hadoop.classification.InterfaceStability;
022import org.apache.hadoop.mapred.*;
023
024import java.io.IOException;
025import java.util.Iterator;
026
027/**
028 * The ChainReducer class allows to chain multiple Mapper classes after a
029 * Reducer within the Reducer task.
030 * <p/>
031 * For each record output by the Reducer, the Mapper classes are invoked in a
032 * chained (or piped) fashion, the output of the first becomes the input of the
033 * second, and so on until the last Mapper, the output of the last Mapper will
034 * be written to the task's output.
035 * <p/>
036 * The key functionality of this feature is that the Mappers in the chain do not
037 * need to be aware that they are executed after the Reducer or in a chain.
038 * This enables having reusable specialized Mappers that can be combined to
039 * perform composite operations within a single task.
040 * <p/>
041 * Special care has to be taken when creating chains that the key/values output
042 * by a Mapper are valid for the following Mapper in the chain. It is assumed
043 * all Mappers and the Reduce in the chain use maching output and input key and
044 * value classes as no conversion is done by the chaining code.
045 * <p/>
046 * Using the ChainMapper and the ChainReducer classes is possible to compose
047 * Map/Reduce jobs that look like <code>[MAP+ / REDUCE MAP*]</code>. And
048 * immediate benefit of this pattern is a dramatic reduction in disk IO.
049 * <p/>
050 * IMPORTANT: There is no need to specify the output key/value classes for the
051 * ChainReducer, this is done by the setReducer or the addMapper for the last
052 * element in the chain.
053 * <p/>
054 * ChainReducer usage pattern:
055 * <p/>
056 * <pre>
057 * ...
058 * conf.setJobName("chain");
059 * conf.setInputFormat(TextInputFormat.class);
060 * conf.setOutputFormat(TextOutputFormat.class);
061 * <p/>
062 * JobConf mapAConf = new JobConf(false);
063 * ...
064 * ChainMapper.addMapper(conf, AMap.class, LongWritable.class, Text.class,
065 *   Text.class, Text.class, true, mapAConf);
066 * <p/>
067 * JobConf mapBConf = new JobConf(false);
068 * ...
069 * ChainMapper.addMapper(conf, BMap.class, Text.class, Text.class,
070 *   LongWritable.class, Text.class, false, mapBConf);
071 * <p/>
072 * JobConf reduceConf = new JobConf(false);
073 * ...
074 * ChainReducer.setReducer(conf, XReduce.class, LongWritable.class, Text.class,
075 *   Text.class, Text.class, true, reduceConf);
076 * <p/>
077 * ChainReducer.addMapper(conf, CMap.class, Text.class, Text.class,
078 *   LongWritable.class, Text.class, false, null);
079 * <p/>
080 * ChainReducer.addMapper(conf, DMap.class, LongWritable.class, Text.class,
081 *   LongWritable.class, LongWritable.class, true, null);
082 * <p/>
083 * FileInputFormat.setInputPaths(conf, inDir);
084 * FileOutputFormat.setOutputPath(conf, outDir);
085 * ...
086 * <p/>
087 * JobClient jc = new JobClient(conf);
088 * RunningJob job = jc.submitJob(conf);
089 * ...
090 * </pre>
091 */
092@InterfaceAudience.Public
093@InterfaceStability.Stable
094public class ChainReducer implements Reducer {
095
096  /**
097   * Sets the Reducer class to the chain job's JobConf.
098   * <p/>
099   * It has to be specified how key and values are passed from one element of
100   * the chain to the next, by value or by reference. If a Reducer leverages the
101   * assumed semantics that the key and values are not modified by the collector
102   * 'by value' must be used. If the Reducer does not expect this semantics, as
103   * an optimization to avoid serialization and deserialization 'by reference'
104   * can be used.
105   * <p/>
106   * For the added Reducer the configuration given for it,
107   * <code>reducerConf</code>, have precedence over the job's JobConf. This
108   * precedence is in effect when the task is running.
109   * <p/>
110   * IMPORTANT: There is no need to specify the output key/value classes for the
111   * ChainReducer, this is done by the setReducer or the addMapper for the last
112   * element in the chain.
113   *
114   * @param job              job's JobConf to add the Reducer class.
115   * @param klass            the Reducer class to add.
116   * @param inputKeyClass    reducer input key class.
117   * @param inputValueClass  reducer input value class.
118   * @param outputKeyClass   reducer output key class.
119   * @param outputValueClass reducer output value class.
120   * @param byValue          indicates if key/values should be passed by value
121   * to the next Mapper in the chain, if any.
122   * @param reducerConf      a JobConf with the configuration for the Reducer
123   * class. It is recommended to use a JobConf without default values using the
124   * <code>JobConf(boolean loadDefaults)</code> constructor with FALSE.
125   */
126  public static <K1, V1, K2, V2> void setReducer(JobConf job,
127                           Class<? extends Reducer<K1, V1, K2, V2>> klass,
128                           Class<? extends K1> inputKeyClass,
129                           Class<? extends V1> inputValueClass,
130                           Class<? extends K2> outputKeyClass,
131                           Class<? extends V2> outputValueClass,
132                           boolean byValue, JobConf reducerConf) {
133    job.setReducerClass(ChainReducer.class);
134    job.setOutputKeyClass(outputKeyClass);
135    job.setOutputValueClass(outputValueClass);
136    Chain.setReducer(job, klass, inputKeyClass, inputValueClass, outputKeyClass,
137                     outputValueClass, byValue, reducerConf);
138  }
139
140  /**
141   * Adds a Mapper class to the chain job's JobConf.
142   * <p/>
143   * It has to be specified how key and values are passed from one element of
144   * the chain to the next, by value or by reference. If a Mapper leverages the
145   * assumed semantics that the key and values are not modified by the collector
146   * 'by value' must be used. If the Mapper does not expect this semantics, as
147   * an optimization to avoid serialization and deserialization 'by reference'
148   * can be used.
149   * <p/>
150   * For the added Mapper the configuration given for it,
151   * <code>mapperConf</code>, have precedence over the job's JobConf. This
152   * precedence is in effect when the task is running.
153   * <p/>
154   * IMPORTANT: There is no need to specify the output key/value classes for the
155   * ChainMapper, this is done by the addMapper for the last mapper in the chain
156   * .
157   *
158   * @param job              chain job's JobConf to add the Mapper class.
159   * @param klass            the Mapper class to add.
160   * @param inputKeyClass    mapper input key class.
161   * @param inputValueClass  mapper input value class.
162   * @param outputKeyClass   mapper output key class.
163   * @param outputValueClass mapper output value class.
164   * @param byValue          indicates if key/values should be passed by value
165   * to the next Mapper in the chain, if any.
166   * @param mapperConf       a JobConf with the configuration for the Mapper
167   * class. It is recommended to use a JobConf without default values using the
168   * <code>JobConf(boolean loadDefaults)</code> constructor with FALSE.
169   */
170  public static <K1, V1, K2, V2> void addMapper(JobConf job,
171                           Class<? extends Mapper<K1, V1, K2, V2>> klass,
172                           Class<? extends K1> inputKeyClass,
173                           Class<? extends V1> inputValueClass,
174                           Class<? extends K2> outputKeyClass,
175                           Class<? extends V2> outputValueClass,
176                           boolean byValue, JobConf mapperConf) {
177    job.setOutputKeyClass(outputKeyClass);
178    job.setOutputValueClass(outputValueClass);
179    Chain.addMapper(false, job, klass, inputKeyClass, inputValueClass,
180                    outputKeyClass, outputValueClass, byValue, mapperConf);
181  }
182
183  private Chain chain;
184
185  /**
186   * Constructor.
187   */
188  public ChainReducer() {
189    chain = new Chain(false);
190  }
191
192  /**
193   * Configures the ChainReducer, the Reducer and all the Mappers in the chain.
194   * <p/>
195   * If this method is overriden <code>super.configure(...)</code> should be
196   * invoked at the beginning of the overwriter method.
197   */
198  public void configure(JobConf job) {
199    chain.configure(job);
200  }
201
202  /**
203   * Chains the <code>reduce(...)</code> method of the Reducer with the
204   * <code>map(...) </code> methods of the Mappers in the chain.
205   */
206  @SuppressWarnings({"unchecked"})
207  public void reduce(Object key, Iterator values, OutputCollector output,
208                     Reporter reporter) throws IOException {
209    Reducer reducer = chain.getReducer();
210    if (reducer != null) {
211      reducer.reduce(key, values, chain.getReducerCollector(output, reporter),
212                     reporter);
213    }
214  }
215
216  /**
217   * Closes  the ChainReducer, the Reducer and all the Mappers in the chain.
218   * <p/>
219   * If this method is overriden <code>super.close()</code> should be
220   * invoked at the end of the overwriter method.
221   */
222  public void close() throws IOException {
223    chain.close();
224  }
225
226}