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
019package org.apache.hadoop.mapred.lib.aggregate;
020
021import java.io.IOException;
022import java.util.ArrayList;
023
024import org.apache.hadoop.classification.InterfaceAudience;
025import org.apache.hadoop.classification.InterfaceStability;
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.fs.Path;
028import org.apache.hadoop.io.Text;
029import org.apache.hadoop.mapred.FileInputFormat;
030import org.apache.hadoop.mapred.FileOutputFormat;
031import org.apache.hadoop.mapred.InputFormat;
032import org.apache.hadoop.mapred.JobClient;
033import org.apache.hadoop.mapred.JobConf;
034import org.apache.hadoop.mapred.SequenceFileInputFormat;
035import org.apache.hadoop.mapred.TextInputFormat;
036import org.apache.hadoop.mapred.TextOutputFormat;
037import org.apache.hadoop.mapred.jobcontrol.Job;
038import org.apache.hadoop.mapred.jobcontrol.JobControl;
039import org.apache.hadoop.util.GenericOptionsParser;
040
041/**
042 * This is the main class for creating a map/reduce job using Aggregate
043 * framework. The Aggregate is a specialization of map/reduce framework,
044 * specilizing for performing various simple aggregations.
045 * 
046 * Generally speaking, in order to implement an application using Map/Reduce
047 * model, the developer is to implement Map and Reduce functions (and possibly
048 * combine function). However, a lot of applications related to counting and
049 * statistics computing have very similar characteristics. Aggregate abstracts
050 * out the general patterns of these functions and implementing those patterns.
051 * In particular, the package provides generic mapper/redducer/combiner classes,
052 * and a set of built-in value aggregators, and a generic utility class that
053 * helps user create map/reduce jobs using the generic class. The built-in
054 * aggregators include:
055 * 
056 * sum over numeric values count the number of distinct values compute the
057 * histogram of values compute the minimum, maximum, media,average, standard
058 * deviation of numeric values
059 * 
060 * The developer using Aggregate will need only to provide a plugin class
061 * conforming to the following interface:
062 * 
063 * public interface ValueAggregatorDescriptor { public ArrayList<Entry>
064 * generateKeyValPairs(Object key, Object value); public void
065 * configure(JobConfjob); }
066 * 
067 * The package also provides a base class, ValueAggregatorBaseDescriptor,
068 * implementing the above interface. The user can extend the base class and
069 * implement generateKeyValPairs accordingly.
070 * 
071 * The primary work of generateKeyValPairs is to emit one or more key/value
072 * pairs based on the input key/value pair. The key in an output key/value pair
073 * encode two pieces of information: aggregation type and aggregation id. The
074 * value will be aggregated onto the aggregation id according the aggregation
075 * type.
076 * 
077 * This class offers a function to generate a map/reduce job using Aggregate
078 * framework. The function takes the following parameters: input directory spec
079 * input format (text or sequence file) output directory a file specifying the
080 * user plugin class
081 */
082@InterfaceAudience.Public
083@InterfaceStability.Stable
084public class ValueAggregatorJob {
085
086  public static JobControl createValueAggregatorJobs(String args[]
087    , Class<? extends ValueAggregatorDescriptor>[] descriptors) throws IOException {
088    
089    JobControl theControl = new JobControl("ValueAggregatorJobs");
090    ArrayList<Job> dependingJobs = new ArrayList<Job>();
091    JobConf aJobConf = createValueAggregatorJob(args);
092    if(descriptors != null)
093      setAggregatorDescriptors(aJobConf, descriptors);
094    Job aJob = new Job(aJobConf, dependingJobs);
095    theControl.addJob(aJob);
096    return theControl;
097  }
098
099  public static JobControl createValueAggregatorJobs(String args[]) throws IOException {
100    return createValueAggregatorJobs(args, null);
101  }
102  
103  /**
104   * Create an Aggregate based map/reduce job.
105   *
106   * @param args the arguments used for job creation. Generic hadoop
107   * arguments are accepted.
108   * @param caller the the caller class.
109   * @return a JobConf object ready for submission.
110   *
111   * @throws IOException
112   * @see GenericOptionsParser
113   */
114  @SuppressWarnings("rawtypes")
115  public static JobConf createValueAggregatorJob(String args[], Class<?> caller)
116    throws IOException {
117
118    Configuration conf = new Configuration();
119    
120    GenericOptionsParser genericParser 
121      = new GenericOptionsParser(conf, args);
122    args = genericParser.getRemainingArgs();
123    
124    if (args.length < 2) {
125      System.out.println("usage: inputDirs outDir "
126          + "[numOfReducer [textinputformat|seq [specfile [jobName]]]]");
127      GenericOptionsParser.printGenericCommandUsage(System.out);
128      System.exit(1);
129    }
130    String inputDir = args[0];
131    String outputDir = args[1];
132    int numOfReducers = 1;
133    if (args.length > 2) {
134      numOfReducers = Integer.parseInt(args[2]);
135    }
136
137    Class<? extends InputFormat> theInputFormat =
138      TextInputFormat.class;
139    if (args.length > 3 && 
140        args[3].compareToIgnoreCase("textinputformat") == 0) {
141      theInputFormat = TextInputFormat.class;
142    } else {
143      theInputFormat = SequenceFileInputFormat.class;
144    }
145
146    Path specFile = null;
147
148    if (args.length > 4) {
149      specFile = new Path(args[4]);
150    }
151
152    String jobName = "";
153    
154    if (args.length > 5) {
155      jobName = args[5];
156    }
157    
158    JobConf theJob = new JobConf(conf);
159    if (specFile != null) {
160      theJob.addResource(specFile);
161    }
162    String userJarFile = theJob.get("user.jar.file");
163    if (userJarFile == null) {
164      theJob.setJarByClass(caller != null ? caller : ValueAggregatorJob.class);
165    } else {
166      theJob.setJar(userJarFile);
167    }
168    theJob.setJobName("ValueAggregatorJob: " + jobName);
169
170    FileInputFormat.addInputPaths(theJob, inputDir);
171
172    theJob.setInputFormat(theInputFormat);
173    
174    theJob.setMapperClass(ValueAggregatorMapper.class);
175    FileOutputFormat.setOutputPath(theJob, new Path(outputDir));
176    theJob.setOutputFormat(TextOutputFormat.class);
177    theJob.setMapOutputKeyClass(Text.class);
178    theJob.setMapOutputValueClass(Text.class);
179    theJob.setOutputKeyClass(Text.class);
180    theJob.setOutputValueClass(Text.class);
181    theJob.setReducerClass(ValueAggregatorReducer.class);
182    theJob.setCombinerClass(ValueAggregatorCombiner.class);
183    theJob.setNumMapTasks(1);
184    theJob.setNumReduceTasks(numOfReducers);
185    return theJob;
186  }
187
188  /**
189   * Create an Aggregate based map/reduce job.
190   * 
191   * @param args the arguments used for job creation. Generic hadoop
192   * arguments are accepted.
193   * @return a JobConf object ready for submission.
194   * 
195   * @throws IOException
196   * @see GenericOptionsParser
197   */
198  public static JobConf createValueAggregatorJob(String args[])
199    throws IOException {
200    return createValueAggregatorJob(args, ValueAggregator.class);
201  }
202
203  public static JobConf createValueAggregatorJob(String args[]
204    , Class<? extends ValueAggregatorDescriptor>[] descriptors)
205  throws IOException {
206    JobConf job = createValueAggregatorJob(args);
207    setAggregatorDescriptors(job, descriptors);
208    return job;
209  }
210  
211  public static void setAggregatorDescriptors(JobConf job
212      , Class<? extends ValueAggregatorDescriptor>[] descriptors) {
213    job.setInt("aggregator.descriptor.num", descriptors.length);
214    //specify the aggregator descriptors
215    for(int i=0; i< descriptors.length; i++) {
216      job.set("aggregator.descriptor." + i, "UserDefined," + descriptors[i].getName());
217    }    
218  }
219
220  public static JobConf createValueAggregatorJob(String args[],
221      Class<? extends ValueAggregatorDescriptor>[] descriptors,
222      Class<?> caller) throws IOException {
223    JobConf job = createValueAggregatorJob(args, caller);
224    setAggregatorDescriptors(job, descriptors);
225    return job;
226  }
227
228  /**
229   * create and run an Aggregate based map/reduce job.
230   * 
231   * @param args the arguments used for job creation
232   * @throws IOException
233   */
234  public static void main(String args[]) throws IOException {
235    JobConf job = ValueAggregatorJob.createValueAggregatorJob(args);
236    JobClient.runJob(job);
237  }
238}