MAPREDUCE-7174. Mapreduce example wordmedian should handle generic options. Contributed by Fei Hui.
(cherry picked from commit 9e7196749e
)
This commit is contained in:
parent
32fc4ac752
commit
2a5b2eda7c
|
@ -35,6 +35,7 @@ import org.apache.hadoop.mapreduce.Reducer;
|
|||
import org.apache.hadoop.mapreduce.TaskCounter;
|
||||
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||
import org.apache.hadoop.util.GenericOptionsParser;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
|
||||
|
@ -173,13 +174,15 @@ public class WordMedian extends Configured implements Tool {
|
|||
|
||||
@Override
|
||||
public int run(String[] args) throws Exception {
|
||||
if (args.length != 2) {
|
||||
Configuration conf = new Configuration();
|
||||
String[] otherArgs =
|
||||
new GenericOptionsParser(conf, args).getRemainingArgs();
|
||||
if (otherArgs.length != 2) {
|
||||
System.err.println("Usage: wordmedian <in> <out>");
|
||||
return 0;
|
||||
}
|
||||
|
||||
setConf(new Configuration());
|
||||
Configuration conf = getConf();
|
||||
setConf(conf);
|
||||
|
||||
Job job = Job.getInstance(conf, "word median");
|
||||
job.setJarByClass(WordMedian.class);
|
||||
|
@ -188,8 +191,8 @@ public class WordMedian extends Configured implements Tool {
|
|||
job.setReducerClass(WordMedianReducer.class);
|
||||
job.setOutputKeyClass(IntWritable.class);
|
||||
job.setOutputValueClass(IntWritable.class);
|
||||
FileInputFormat.addInputPath(job, new Path(args[0]));
|
||||
FileOutputFormat.setOutputPath(job, new Path(args[1]));
|
||||
FileInputFormat.addInputPath(job, new Path(otherArgs[0]));
|
||||
FileOutputFormat.setOutputPath(job, new Path(otherArgs[1]));
|
||||
boolean result = job.waitForCompletion(true);
|
||||
|
||||
// Wait for JOB 1 -- get middle value to check for Median
|
||||
|
|
Loading…
Reference in New Issue