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    
019    package org.apache.hadoop.mapreduce.lib.aggregate;
020    
021    import java.io.IOException;
022    import java.util.ArrayList;
023    
024    import org.apache.hadoop.classification.InterfaceAudience;
025    import org.apache.hadoop.classification.InterfaceStability;
026    import org.apache.hadoop.conf.Configuration;
027    import org.apache.hadoop.fs.Path;
028    import org.apache.hadoop.io.Text;
029    import org.apache.hadoop.mapreduce.InputFormat;
030    import org.apache.hadoop.mapreduce.Job;
031    import org.apache.hadoop.mapreduce.MRJobConfig;
032    import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
033    import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
034    import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
035    import org.apache.hadoop.mapreduce.lib.jobcontrol.ControlledJob;
036    import org.apache.hadoop.mapreduce.lib.jobcontrol.JobControl;
037    import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
038    import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
039    import 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     * specializing 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 
052     * classes, and a set of built-in value aggregators, and a generic utility 
053     * class that helps user create map/reduce jobs using the generic class. 
054     * The built-in 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(Configuration conf); }
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     */
083    @InterfaceAudience.Public
084    @InterfaceStability.Stable
085    public class ValueAggregatorJob {
086    
087      public static JobControl createValueAggregatorJobs(String args[],
088        Class<? extends ValueAggregatorDescriptor>[] descriptors) 
089      throws IOException {
090        
091        JobControl theControl = new JobControl("ValueAggregatorJobs");
092        ArrayList<ControlledJob> dependingJobs = new ArrayList<ControlledJob>();
093        Configuration conf = new Configuration();
094        if (descriptors != null) {
095          conf = setAggregatorDescriptors(descriptors);
096        }
097        Job job = createValueAggregatorJob(conf, args);
098        ControlledJob cjob = new ControlledJob(job, dependingJobs);
099        theControl.addJob(cjob);
100        return theControl;
101      }
102    
103      public static JobControl createValueAggregatorJobs(String args[]) 
104          throws IOException {
105        return createValueAggregatorJobs(args, null);
106      }
107      
108      /**
109       * Create an Aggregate based map/reduce job.
110       * 
111       * @param conf The configuration for job
112       * @param args the arguments used for job creation. Generic hadoop
113       * arguments are accepted.
114       * @return a Job object ready for submission.
115       * 
116       * @throws IOException
117       * @see GenericOptionsParser
118       */
119      public static Job createValueAggregatorJob(Configuration conf, String args[])
120          throws IOException {
121    
122        GenericOptionsParser genericParser 
123          = new GenericOptionsParser(conf, args);
124        args = genericParser.getRemainingArgs();
125        
126        if (args.length < 2) {
127          System.out.println("usage: inputDirs outDir "
128              + "[numOfReducer [textinputformat|seq [specfile [jobName]]]]");
129          GenericOptionsParser.printGenericCommandUsage(System.out);
130          System.exit(2);
131        }
132        String inputDir = args[0];
133        String outputDir = args[1];
134        int numOfReducers = 1;
135        if (args.length > 2) {
136          numOfReducers = Integer.parseInt(args[2]);
137        }
138    
139        Class<? extends InputFormat> theInputFormat = null;
140        if (args.length > 3 && 
141            args[3].compareToIgnoreCase("textinputformat") == 0) {
142          theInputFormat = TextInputFormat.class;
143        } else {
144          theInputFormat = SequenceFileInputFormat.class;
145        }
146    
147        Path specFile = null;
148    
149        if (args.length > 4) {
150          specFile = new Path(args[4]);
151        }
152    
153        String jobName = "";
154        
155        if (args.length > 5) {
156          jobName = args[5];
157        }
158    
159        if (specFile != null) {
160          conf.addResource(specFile);
161        }
162        String userJarFile = conf.get(ValueAggregatorJobBase.USER_JAR);
163        if (userJarFile != null) {
164          conf.set(MRJobConfig.JAR, userJarFile);
165        }
166    
167        Job theJob = new Job(conf);
168        if (userJarFile == null) {
169          theJob.setJarByClass(ValueAggregator.class);
170        } 
171        theJob.setJobName("ValueAggregatorJob: " + jobName);
172    
173        FileInputFormat.addInputPaths(theJob, inputDir);
174    
175        theJob.setInputFormatClass(theInputFormat);
176        
177        theJob.setMapperClass(ValueAggregatorMapper.class);
178        FileOutputFormat.setOutputPath(theJob, new Path(outputDir));
179        theJob.setOutputFormatClass(TextOutputFormat.class);
180        theJob.setMapOutputKeyClass(Text.class);
181        theJob.setMapOutputValueClass(Text.class);
182        theJob.setOutputKeyClass(Text.class);
183        theJob.setOutputValueClass(Text.class);
184        theJob.setReducerClass(ValueAggregatorReducer.class);
185        theJob.setCombinerClass(ValueAggregatorCombiner.class);
186        theJob.setNumReduceTasks(numOfReducers);
187        return theJob;
188      }
189    
190      public static Job createValueAggregatorJob(String args[], 
191          Class<? extends ValueAggregatorDescriptor>[] descriptors) 
192          throws IOException {
193        return createValueAggregatorJob(
194                 setAggregatorDescriptors(descriptors), args);
195      }
196      
197      public static Configuration setAggregatorDescriptors(
198          Class<? extends ValueAggregatorDescriptor>[] descriptors) {
199        Configuration conf = new Configuration();
200        conf.setInt(ValueAggregatorJobBase.DESCRIPTOR_NUM, descriptors.length);
201        //specify the aggregator descriptors
202        for(int i=0; i< descriptors.length; i++) {
203          conf.set(ValueAggregatorJobBase.DESCRIPTOR + i, 
204                   "UserDefined," + descriptors[i].getName());
205        }
206        return conf;
207      }
208      
209      /**
210       * create and run an Aggregate based map/reduce job.
211       * 
212       * @param args the arguments used for job creation
213       * @throws IOException
214       */
215      public static void main(String args[]) 
216          throws IOException, InterruptedException, ClassNotFoundException {
217        Job job = ValueAggregatorJob.createValueAggregatorJob(
218                    new Configuration(), args);
219        int ret = job.waitForCompletion(true) ? 0 : 1;
220        System.exit(ret);
221      }
222    }