|
@@ -21,6 +21,8 @@ package org.apache.hadoop.mapred;
|
|
|
import java.io.DataInput;
|
|
|
import java.io.DataOutput;
|
|
|
import java.io.IOException;
|
|
|
+import java.lang.reflect.Constructor;
|
|
|
+import java.lang.reflect.InvocationTargetException;
|
|
|
import java.text.NumberFormat;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Iterator;
|
|
@@ -41,6 +43,7 @@ import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.io.Writable;
|
|
|
import org.apache.hadoop.io.serializer.Deserializer;
|
|
|
import org.apache.hadoop.io.serializer.SerializationFactory;
|
|
|
+import org.apache.hadoop.mapred.Counters.Counter;
|
|
|
import org.apache.hadoop.mapred.IFile.Writer;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.util.Progress;
|
|
@@ -986,4 +989,208 @@ abstract class Task implements Writable, Configurable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private static final Constructor<org.apache.hadoop.mapreduce.Reducer.Context>
|
|
|
+ contextConstructor;
|
|
|
+ static {
|
|
|
+ try {
|
|
|
+ contextConstructor =
|
|
|
+ org.apache.hadoop.mapreduce.Reducer.Context.class.getConstructor
|
|
|
+ (new Class[]{org.apache.hadoop.mapreduce.Reducer.class,
|
|
|
+ Configuration.class,
|
|
|
+ org.apache.hadoop.mapreduce.TaskAttemptID.class,
|
|
|
+ RawKeyValueIterator.class,
|
|
|
+ org.apache.hadoop.mapreduce.Counter.class,
|
|
|
+ org.apache.hadoop.mapreduce.RecordWriter.class,
|
|
|
+ org.apache.hadoop.mapreduce.OutputCommitter.class,
|
|
|
+ org.apache.hadoop.mapreduce.StatusReporter.class,
|
|
|
+ RawComparator.class,
|
|
|
+ Class.class,
|
|
|
+ Class.class});
|
|
|
+ } catch (NoSuchMethodException nme) {
|
|
|
+ throw new IllegalArgumentException("Can't find constructor");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ protected static <INKEY,INVALUE,OUTKEY,OUTVALUE>
|
|
|
+ org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context
|
|
|
+ createReduceContext(org.apache.hadoop.mapreduce.Reducer
|
|
|
+ <INKEY,INVALUE,OUTKEY,OUTVALUE> reducer,
|
|
|
+ Configuration job,
|
|
|
+ org.apache.hadoop.mapreduce.TaskAttemptID taskId,
|
|
|
+ RawKeyValueIterator rIter,
|
|
|
+ org.apache.hadoop.mapreduce.Counter inputCounter,
|
|
|
+ org.apache.hadoop.mapreduce.RecordWriter<OUTKEY,OUTVALUE> output,
|
|
|
+ org.apache.hadoop.mapreduce.OutputCommitter committer,
|
|
|
+ org.apache.hadoop.mapreduce.StatusReporter reporter,
|
|
|
+ RawComparator<INKEY> comparator,
|
|
|
+ Class<INKEY> keyClass, Class<INVALUE> valueClass
|
|
|
+ ) throws IOException, ClassNotFoundException {
|
|
|
+ try {
|
|
|
+
|
|
|
+ return contextConstructor.newInstance(reducer, job, taskId,
|
|
|
+ rIter, inputCounter, output,
|
|
|
+ committer, reporter, comparator,
|
|
|
+ keyClass, valueClass);
|
|
|
+ } catch (InstantiationException e) {
|
|
|
+ throw new IOException("Can't create Context", e);
|
|
|
+ } catch (InvocationTargetException e) {
|
|
|
+ throw new IOException("Can't invoke Context constructor", e);
|
|
|
+ } catch (IllegalAccessException e) {
|
|
|
+ throw new IOException("Can't invoke Context constructor", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ protected static abstract class CombinerRunner<K,V> {
|
|
|
+ protected final Counters.Counter inputCounter;
|
|
|
+ protected final JobConf job;
|
|
|
+ protected final TaskReporter reporter;
|
|
|
+
|
|
|
+ CombinerRunner(Counters.Counter inputCounter,
|
|
|
+ JobConf job,
|
|
|
+ TaskReporter reporter) {
|
|
|
+ this.inputCounter = inputCounter;
|
|
|
+ this.job = job;
|
|
|
+ this.reporter = reporter;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Run the combiner over a set of inputs.
|
|
|
+ * @param iterator the key/value pairs to use as input
|
|
|
+ * @param collector the output collector
|
|
|
+ */
|
|
|
+ abstract void combine(RawKeyValueIterator iterator,
|
|
|
+ OutputCollector<K,V> collector
|
|
|
+ ) throws IOException, InterruptedException,
|
|
|
+ ClassNotFoundException;
|
|
|
+
|
|
|
+ static <K,V>
|
|
|
+ CombinerRunner<K,V> create(JobConf job,
|
|
|
+ TaskAttemptID taskId,
|
|
|
+ Counters.Counter inputCounter,
|
|
|
+ TaskReporter reporter,
|
|
|
+ org.apache.hadoop.mapreduce.OutputCommitter committer
|
|
|
+ ) throws ClassNotFoundException {
|
|
|
+ Class<? extends Reducer<K,V,K,V>> cls =
|
|
|
+ (Class<? extends Reducer<K,V,K,V>>) job.getCombinerClass();
|
|
|
+ if (cls != null) {
|
|
|
+ return new OldCombinerRunner(cls, job, inputCounter, reporter);
|
|
|
+ }
|
|
|
+ // make a task context so we can get the classes
|
|
|
+ org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
|
|
|
+ new org.apache.hadoop.mapreduce.TaskAttemptContext(job, taskId);
|
|
|
+ Class<? extends org.apache.hadoop.mapreduce.Reducer<K,V,K,V>> newcls =
|
|
|
+ (Class<? extends org.apache.hadoop.mapreduce.Reducer<K,V,K,V>>)
|
|
|
+ taskContext.getCombinerClass();
|
|
|
+ if (newcls != null) {
|
|
|
+ return new NewCombinerRunner<K,V>(newcls, job, taskId, taskContext,
|
|
|
+ inputCounter, reporter, committer);
|
|
|
+ }
|
|
|
+
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ protected static class OldCombinerRunner<K,V> extends CombinerRunner<K,V> {
|
|
|
+ private final Class<? extends Reducer<K,V,K,V>> combinerClass;
|
|
|
+ private final Class<K> keyClass;
|
|
|
+ private final Class<V> valueClass;
|
|
|
+ private final RawComparator<K> comparator;
|
|
|
+
|
|
|
+ protected OldCombinerRunner(Class<? extends Reducer<K,V,K,V>> cls,
|
|
|
+ JobConf conf,
|
|
|
+ Counters.Counter inputCounter,
|
|
|
+ TaskReporter reporter) {
|
|
|
+ super(inputCounter, conf, reporter);
|
|
|
+ combinerClass = cls;
|
|
|
+ keyClass = (Class<K>) job.getMapOutputKeyClass();
|
|
|
+ valueClass = (Class<V>) job.getMapOutputValueClass();
|
|
|
+ comparator = (RawComparator<K>) job.getOutputKeyComparator();
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ protected void combine(RawKeyValueIterator kvIter,
|
|
|
+ OutputCollector<K,V> combineCollector
|
|
|
+ ) throws IOException {
|
|
|
+ Reducer<K,V,K,V> combiner =
|
|
|
+ ReflectionUtils.newInstance(combinerClass, job);
|
|
|
+ try {
|
|
|
+ CombineValuesIterator<K,V> values =
|
|
|
+ new CombineValuesIterator<K,V>(kvIter, comparator, keyClass,
|
|
|
+ valueClass, job, Reporter.NULL,
|
|
|
+ inputCounter);
|
|
|
+ while (values.more()) {
|
|
|
+ combiner.reduce(values.getKey(), values, combineCollector,
|
|
|
+ Reporter.NULL);
|
|
|
+ values.nextKey();
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ combiner.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ protected static class NewCombinerRunner<K, V> extends CombinerRunner<K,V> {
|
|
|
+ private final Class<? extends org.apache.hadoop.mapreduce.Reducer<K,V,K,V>>
|
|
|
+ reducerClass;
|
|
|
+ private final org.apache.hadoop.mapreduce.TaskAttemptID taskId;
|
|
|
+ private final RawComparator<K> comparator;
|
|
|
+ private final Class<K> keyClass;
|
|
|
+ private final Class<V> valueClass;
|
|
|
+ private final org.apache.hadoop.mapreduce.OutputCommitter committer;
|
|
|
+
|
|
|
+ NewCombinerRunner(Class reducerClass,
|
|
|
+ JobConf job,
|
|
|
+ org.apache.hadoop.mapreduce.TaskAttemptID taskId,
|
|
|
+ org.apache.hadoop.mapreduce.TaskAttemptContext context,
|
|
|
+ Counters.Counter inputCounter,
|
|
|
+ TaskReporter reporter,
|
|
|
+ org.apache.hadoop.mapreduce.OutputCommitter committer) {
|
|
|
+ super(inputCounter, job, reporter);
|
|
|
+ this.reducerClass = reducerClass;
|
|
|
+ this.taskId = taskId;
|
|
|
+ keyClass = (Class<K>) context.getMapOutputKeyClass();
|
|
|
+ valueClass = (Class<V>) context.getMapOutputValueClass();
|
|
|
+ comparator = (RawComparator<K>) context.getSortComparator();
|
|
|
+ this.committer = committer;
|
|
|
+ }
|
|
|
+
|
|
|
+ private static class OutputConverter<K,V>
|
|
|
+ extends org.apache.hadoop.mapreduce.RecordWriter<K,V> {
|
|
|
+ OutputCollector<K,V> output;
|
|
|
+ OutputConverter(OutputCollector<K,V> output) {
|
|
|
+ this.output = output;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void close(org.apache.hadoop.mapreduce.TaskAttemptContext context){
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void write(K key, V value
|
|
|
+ ) throws IOException, InterruptedException {
|
|
|
+ output.collect(key,value);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ void combine(RawKeyValueIterator iterator,
|
|
|
+ OutputCollector<K,V> collector
|
|
|
+ ) throws IOException, InterruptedException,
|
|
|
+ ClassNotFoundException {
|
|
|
+ // make a reducer
|
|
|
+ org.apache.hadoop.mapreduce.Reducer<K,V,K,V> reducer =
|
|
|
+ (org.apache.hadoop.mapreduce.Reducer<K,V,K,V>)
|
|
|
+ ReflectionUtils.newInstance(reducerClass, job);
|
|
|
+ org.apache.hadoop.mapreduce.Reducer.Context
|
|
|
+ reducerContext = createReduceContext(reducer, job, taskId,
|
|
|
+ iterator, inputCounter,
|
|
|
+ new OutputConverter(collector),
|
|
|
+ committer,
|
|
|
+ reporter, comparator, keyClass,
|
|
|
+ valueClass);
|
|
|
+ reducer.run(reducerContext);
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
}
|