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 */ 018 package org.apache.hadoop.mapred.lib; 019 020 import org.apache.hadoop.classification.InterfaceAudience; 021 import org.apache.hadoop.classification.InterfaceStability; 022 import org.apache.hadoop.mapred.*; 023 024 import java.io.IOException; 025 import 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 * @deprecated 092 * Use {@link org.apache.hadoop.mapreduce.lib.chain.ChainReducer} instead 093 */ 094 @Deprecated 095 @InterfaceAudience.Public 096 @InterfaceStability.Stable 097 public class ChainReducer implements Reducer { 098 099 /** 100 * Sets the Reducer class to the chain job's JobConf. 101 * <p/> 102 * It has to be specified how key and values are passed from one element of 103 * the chain to the next, by value or by reference. If a Reducer leverages the 104 * assumed semantics that the key and values are not modified by the collector 105 * 'by value' must be used. If the Reducer does not expect this semantics, as 106 * an optimization to avoid serialization and deserialization 'by reference' 107 * can be used. 108 * <p/> 109 * For the added Reducer the configuration given for it, 110 * <code>reducerConf</code>, have precedence over the job's JobConf. This 111 * precedence is in effect when the task is running. 112 * <p/> 113 * IMPORTANT: There is no need to specify the output key/value classes for the 114 * ChainReducer, this is done by the setReducer or the addMapper for the last 115 * element in the chain. 116 * 117 * @param job job's JobConf to add the Reducer class. 118 * @param klass the Reducer class to add. 119 * @param inputKeyClass reducer input key class. 120 * @param inputValueClass reducer input value class. 121 * @param outputKeyClass reducer output key class. 122 * @param outputValueClass reducer output value class. 123 * @param byValue indicates if key/values should be passed by value 124 * to the next Mapper in the chain, if any. 125 * @param reducerConf a JobConf with the configuration for the Reducer 126 * class. It is recommended to use a JobConf without default values using the 127 * <code>JobConf(boolean loadDefaults)</code> constructor with FALSE. 128 */ 129 public static <K1, V1, K2, V2> void setReducer(JobConf job, 130 Class<? extends Reducer<K1, V1, K2, V2>> klass, 131 Class<? extends K1> inputKeyClass, 132 Class<? extends V1> inputValueClass, 133 Class<? extends K2> outputKeyClass, 134 Class<? extends V2> outputValueClass, 135 boolean byValue, JobConf reducerConf) { 136 job.setReducerClass(ChainReducer.class); 137 job.setOutputKeyClass(outputKeyClass); 138 job.setOutputValueClass(outputValueClass); 139 Chain.setReducer(job, klass, inputKeyClass, inputValueClass, outputKeyClass, 140 outputValueClass, byValue, reducerConf); 141 } 142 143 /** 144 * Adds a Mapper class to the chain job's JobConf. 145 * <p/> 146 * It has to be specified how key and values are passed from one element of 147 * the chain to the next, by value or by reference. If a Mapper leverages the 148 * assumed semantics that the key and values are not modified by the collector 149 * 'by value' must be used. If the Mapper does not expect this semantics, as 150 * an optimization to avoid serialization and deserialization 'by reference' 151 * can be used. 152 * <p/> 153 * For the added Mapper the configuration given for it, 154 * <code>mapperConf</code>, have precedence over the job's JobConf. This 155 * precedence is in effect when the task is running. 156 * <p/> 157 * IMPORTANT: There is no need to specify the output key/value classes for the 158 * ChainMapper, this is done by the addMapper for the last mapper in the chain 159 * . 160 * 161 * @param job chain job's JobConf to add the Mapper class. 162 * @param klass the Mapper class to add. 163 * @param inputKeyClass mapper input key class. 164 * @param inputValueClass mapper input value class. 165 * @param outputKeyClass mapper output key class. 166 * @param outputValueClass mapper output value class. 167 * @param byValue indicates if key/values should be passed by value 168 * to the next Mapper in the chain, if any. 169 * @param mapperConf a JobConf with the configuration for the Mapper 170 * class. It is recommended to use a JobConf without default values using the 171 * <code>JobConf(boolean loadDefaults)</code> constructor with FALSE. 172 */ 173 public static <K1, V1, K2, V2> void addMapper(JobConf job, 174 Class<? extends Mapper<K1, V1, K2, V2>> klass, 175 Class<? extends K1> inputKeyClass, 176 Class<? extends V1> inputValueClass, 177 Class<? extends K2> outputKeyClass, 178 Class<? extends V2> outputValueClass, 179 boolean byValue, JobConf mapperConf) { 180 job.setOutputKeyClass(outputKeyClass); 181 job.setOutputValueClass(outputValueClass); 182 Chain.addMapper(false, job, klass, inputKeyClass, inputValueClass, 183 outputKeyClass, outputValueClass, byValue, mapperConf); 184 } 185 186 private Chain chain; 187 188 /** 189 * Constructor. 190 */ 191 public ChainReducer() { 192 chain = new Chain(false); 193 } 194 195 /** 196 * Configures the ChainReducer, the Reducer and all the Mappers in the chain. 197 * <p/> 198 * If this method is overriden <code>super.configure(...)</code> should be 199 * invoked at the beginning of the overwriter method. 200 */ 201 public void configure(JobConf job) { 202 chain.configure(job); 203 } 204 205 /** 206 * Chains the <code>reduce(...)</code> method of the Reducer with the 207 * <code>map(...) </code> methods of the Mappers in the chain. 208 */ 209 @SuppressWarnings({"unchecked"}) 210 public void reduce(Object key, Iterator values, OutputCollector output, 211 Reporter reporter) throws IOException { 212 Reducer reducer = chain.getReducer(); 213 if (reducer != null) { 214 reducer.reduce(key, values, chain.getReducerCollector(output, reporter), 215 reporter); 216 } 217 } 218 219 /** 220 * Closes the ChainReducer, the Reducer and all the Mappers in the chain. 221 * <p/> 222 * If this method is overriden <code>super.close()</code> should be 223 * invoked at the end of the overwriter method. 224 */ 225 public void close() throws IOException { 226 chain.close(); 227 } 228 229 }