Merge -c 1397182 from trunk to branch-2 to fix MAPREDUCE-4616. Improve javadoc for MultipleOutputs. Contributed by Tony Burton.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1397183 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0aad6f2dac
commit
267bfc5ff2
|
@ -14,6 +14,9 @@ Release 2.0.3-alpha - Unreleased
|
|||
MAPREDUCE-3678. The Map tasks logs should have the value of input
|
||||
split it processed. (harsh)
|
||||
|
||||
MAPREDUCE-4616. Improve javadoc for MultipleOutputs. (Tony Burton via
|
||||
acmurthy)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
|
|
@ -32,7 +32,10 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
/**
|
||||
* A Convenience class that creates output lazily.
|
||||
* A Convenience class that creates output lazily.
|
||||
* Use in conjuction with org.apache.hadoop.mapreduce.lib.output.MultipleOutputs to recreate the
|
||||
* behaviour of org.apache.hadoop.mapred.lib.MultipleTextOutputFormat (etc) of the old Hadoop API.
|
||||
* See {@link MultipleOutputs} documentation for more information.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
|
|
|
@ -20,7 +20,10 @@ package org.apache.hadoop.mapreduce.lib.output;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.mapreduce.Reducer.Context;
|
||||
import org.apache.hadoop.mapreduce.lib.output.MultipleOutputs;
|
||||
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
|
@ -37,6 +40,7 @@ import java.util.*;
|
|||
* Each additional output, or named output, may be configured with its own
|
||||
* <code>OutputFormat</code>, with its own key class and with its own value
|
||||
* class.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Case two: to write data to different files provided by user
|
||||
|
@ -107,6 +111,64 @@ import java.util.*;
|
|||
*
|
||||
* }
|
||||
* </pre>
|
||||
*
|
||||
* <p>
|
||||
* When used in conjuction with org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat,
|
||||
* MultipleOutputs can mimic the behaviour of MultipleTextOutputFormat and MultipleSequenceFileOutputFormat
|
||||
* from the old Hadoop API - ie, output can be written from the Reducer to more than one location.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Use <code>MultipleOutputs.write(KEYOUT key, VALUEOUT value, String baseOutputPath)</code> to write key and
|
||||
* value to a path specified by <code>baseOutputPath</code>, with no need to specify a named output:
|
||||
* </p>
|
||||
*
|
||||
* <pre>
|
||||
* private MultipleOutputs<Text, Text> out;
|
||||
*
|
||||
* public void setup(Context context) {
|
||||
* out = new MultipleOutputs<Text, Text>(context);
|
||||
* ...
|
||||
* }
|
||||
*
|
||||
* public void reduce(Text key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
|
||||
* for (Text t : values) {
|
||||
* out.write(key, t, generateFileName(<<i>parameter list...</i>>));
|
||||
* }
|
||||
* }
|
||||
*
|
||||
* protected void cleanup(Context context) throws IOException, InterruptedException {
|
||||
* out.close();
|
||||
* }
|
||||
* </pre>
|
||||
*
|
||||
* <p>
|
||||
* Use your own code in <code>generateFileName()</code> to create a custom path to your results.
|
||||
* '/' characters in <code>baseOutputPath</code> will be translated into directory levels in your file system.
|
||||
* Also, append your custom-generated path with "part" or similar, otherwise your output will be -00000, -00001 etc.
|
||||
* No call to <code>context.write()</code> is necessary. See example <code>generateFileName()</code> code below.
|
||||
* </p>
|
||||
*
|
||||
* <pre>
|
||||
* private String generateFileName(Text k) {
|
||||
* // expect Text k in format "Surname|Forename"
|
||||
* String[] kStr = k.toString().split("\\|");
|
||||
*
|
||||
* String sName = kStr[0];
|
||||
* String fName = kStr[1];
|
||||
*
|
||||
* // example for k = Smith|John
|
||||
* // output written to /user/hadoop/path/to/output/Smith/John-r-00000 (etc)
|
||||
* return sName + "/" + fName;
|
||||
* }
|
||||
* </pre>
|
||||
*
|
||||
* <p>
|
||||
* Using MultipleOutputs in this way will still create zero-sized default output, eg part-00000.
|
||||
* To prevent this use <code>LazyOutputFormat.setOutputFormatClass(job, TextOutputFormat.class);</code>
|
||||
* instead of <code>job.setOutputFormatClass(TextOutputFormat.class);</code> in your Hadoop job configuration.
|
||||
* </p>
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
|
|
Loading…
Reference in New Issue