|
@@ -19,8 +19,6 @@ package org.apache.hadoop.mapreduce.task.reduce;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
|
|
|
-import org.apache.commons.logging.Log;
|
|
|
-import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
@@ -33,17 +31,17 @@ import org.apache.hadoop.mapred.RawKeyValueIterator;
|
|
|
import org.apache.hadoop.mapred.Reducer;
|
|
|
import org.apache.hadoop.mapred.Reporter;
|
|
|
import org.apache.hadoop.mapred.Task;
|
|
|
+import org.apache.hadoop.mapred.Task.CombineOutputCollector;
|
|
|
import org.apache.hadoop.mapred.TaskStatus;
|
|
|
import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
|
|
|
-import org.apache.hadoop.mapred.Task.CombineOutputCollector;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
|
|
import org.apache.hadoop.util.Progress;
|
|
|
|
|
|
@InterfaceAudience.Private
|
|
|
@InterfaceStability.Unstable
|
|
|
+@SuppressWarnings({"deprecation", "unchecked", "rawtypes"})
|
|
|
public class Shuffle<K, V> implements ExceptionReporter {
|
|
|
- private static final Log LOG = LogFactory.getLog(Shuffle.class);
|
|
|
private static final int PROGRESS_FREQUENCY = 2000;
|
|
|
|
|
|
private final TaskAttemptID reduceId;
|
|
@@ -100,7 +98,6 @@ public class Shuffle<K, V> implements ExceptionReporter {
|
|
|
this, mergePhase, mapOutputFile);
|
|
|
}
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
public RawKeyValueIterator run() throws IOException, InterruptedException {
|
|
|
// Start the map-completion events fetcher thread
|
|
|
final EventFetcher<K,V> eventFetcher =
|
|
@@ -130,19 +127,11 @@ public class Shuffle<K, V> implements ExceptionReporter {
|
|
|
}
|
|
|
|
|
|
// Stop the event-fetcher thread
|
|
|
- eventFetcher.interrupt();
|
|
|
- try {
|
|
|
- eventFetcher.join();
|
|
|
- } catch(Throwable t) {
|
|
|
- LOG.info("Failed to stop " + eventFetcher.getName(), t);
|
|
|
- }
|
|
|
+ eventFetcher.shutDown();
|
|
|
|
|
|
// Stop the map-output fetcher threads
|
|
|
for (Fetcher<K,V> fetcher : fetchers) {
|
|
|
- fetcher.interrupt();
|
|
|
- }
|
|
|
- for (Fetcher<K,V> fetcher : fetchers) {
|
|
|
- fetcher.join();
|
|
|
+ fetcher.shutDown();
|
|
|
}
|
|
|
fetchers = null;
|
|
|
|