|
@@ -26,6 +26,8 @@ import java.util.ArrayList;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
|
|
|
+import org.apache.commons.logging.Log;
|
|
|
+import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
@@ -57,6 +59,7 @@ class ReduceTask extends Task {
|
|
|
});
|
|
|
}
|
|
|
|
|
|
+ private static final Log LOG = LogFactory.getLog(ReduceTask.class.getName());
|
|
|
private int numMaps;
|
|
|
private boolean sortComplete;
|
|
|
|
|
@@ -116,7 +119,6 @@ class ReduceTask extends Task {
|
|
|
private Writable value; // current value
|
|
|
private boolean hasNext; // more w/ this key
|
|
|
private boolean more; // more in file
|
|
|
- private TaskUmbilicalProtocol umbilical;
|
|
|
private WritableComparator comparator;
|
|
|
private Class keyClass;
|
|
|
private Class valClass;
|
|
@@ -124,18 +126,19 @@ class ReduceTask extends Task {
|
|
|
private DataOutputBuffer valOut = new DataOutputBuffer();
|
|
|
private DataInputBuffer valIn = new DataInputBuffer();
|
|
|
private DataInputBuffer keyIn = new DataInputBuffer();
|
|
|
+ protected Reporter reporter;
|
|
|
|
|
|
public ValuesIterator (SequenceFile.Sorter.RawKeyValueIterator in,
|
|
|
WritableComparator comparator, Class keyClass,
|
|
|
- Class valClass, TaskUmbilicalProtocol umbilical,
|
|
|
- Configuration conf)
|
|
|
+ Class valClass, Configuration conf,
|
|
|
+ Reporter reporter)
|
|
|
throws IOException {
|
|
|
this.in = in;
|
|
|
- this.umbilical = umbilical;
|
|
|
this.conf = conf;
|
|
|
this.comparator = comparator;
|
|
|
this.keyClass = keyClass;
|
|
|
this.valClass = valClass;
|
|
|
+ this.reporter = reporter;
|
|
|
getNext();
|
|
|
}
|
|
|
|
|
@@ -144,13 +147,19 @@ class ReduceTask extends Task {
|
|
|
public boolean hasNext() { return hasNext; }
|
|
|
|
|
|
public Object next() {
|
|
|
+ Object result = value; // save value
|
|
|
try {
|
|
|
- Object result = value; // save value
|
|
|
getNext(); // move to next
|
|
|
- return result; // return saved value
|
|
|
} catch (IOException e) {
|
|
|
throw new RuntimeException(e);
|
|
|
}
|
|
|
+ // ignore the error, since failures in progress shouldn't kill us
|
|
|
+ try {
|
|
|
+ reporter.progress();
|
|
|
+ } catch (IOException ie) {
|
|
|
+ LOG.debug("caught exception from progress", ie);
|
|
|
+ }
|
|
|
+ return result; // return saved value
|
|
|
}
|
|
|
|
|
|
public void remove() { throw new RuntimeException("not implemented"); }
|
|
@@ -198,18 +207,20 @@ class ReduceTask extends Task {
|
|
|
}
|
|
|
}
|
|
|
private class ReduceValuesIterator extends ValuesIterator {
|
|
|
- private Reporter reporter;
|
|
|
public ReduceValuesIterator (SequenceFile.Sorter.RawKeyValueIterator in,
|
|
|
WritableComparator comparator, Class keyClass,
|
|
|
- Class valClass, TaskUmbilicalProtocol umbilical,
|
|
|
+ Class valClass,
|
|
|
Configuration conf, Reporter reporter)
|
|
|
throws IOException {
|
|
|
- super(in, comparator, keyClass, valClass, umbilical, conf);
|
|
|
- this.reporter = reporter;
|
|
|
+ super(in, comparator, keyClass, valClass, conf, reporter);
|
|
|
}
|
|
|
public void informReduceProgress() {
|
|
|
reducePhase.set(super.in.getProgress().get()); // update progress
|
|
|
- reportProgress(super.umbilical);
|
|
|
+ try {
|
|
|
+ reporter.progress();
|
|
|
+ } catch (IOException ie) {
|
|
|
+ LOG.debug("Exception caught from progress", ie);
|
|
|
+ }
|
|
|
}
|
|
|
public Object next() {
|
|
|
reporter.incrCounter(REDUCE_INPUT_RECORDS, 1);
|
|
@@ -312,7 +323,7 @@ class ReduceTask extends Task {
|
|
|
Class keyClass = job.getMapOutputKeyClass();
|
|
|
Class valClass = job.getMapOutputValueClass();
|
|
|
ReduceValuesIterator values = new ReduceValuesIterator(rIter, comparator,
|
|
|
- keyClass, valClass, umbilical, job, reporter);
|
|
|
+ keyClass, valClass, job, reporter);
|
|
|
values.informReduceProgress();
|
|
|
while (values.more()) {
|
|
|
reporter.incrCounter(REDUCE_INPUT_GROUPS, 1);
|