|
@@ -17,15 +17,39 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.mapred.lib;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.Collections;
|
|
|
+import java.util.HashMap;
|
|
|
+import java.util.HashSet;
|
|
|
+import java.util.Iterator;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.Set;
|
|
|
+import java.util.StringTokenizer;
|
|
|
+import java.util.concurrent.Callable;
|
|
|
+import java.util.concurrent.ExecutorService;
|
|
|
+import java.util.concurrent.Executors;
|
|
|
+import java.util.concurrent.ThreadFactory;
|
|
|
+import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
+
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
+
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
+import org.apache.hadoop.classification.VisibleForTesting;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
-import org.apache.hadoop.mapred.*;
|
|
|
+import org.apache.hadoop.mapred.FileOutputFormat;
|
|
|
+import org.apache.hadoop.mapred.JobConf;
|
|
|
+import org.apache.hadoop.mapred.OutputCollector;
|
|
|
+import org.apache.hadoop.mapred.OutputFormat;
|
|
|
+import org.apache.hadoop.mapred.RecordWriter;
|
|
|
+import org.apache.hadoop.mapred.Reporter;
|
|
|
+import org.apache.hadoop.mapreduce.MRConfig;
|
|
|
import org.apache.hadoop.util.Progressable;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
-import java.util.*;
|
|
|
-
|
|
|
/**
|
|
|
* The MultipleOutputs class simplifies writing to additional outputs other
|
|
|
* than the job default output via the <code>OutputCollector</code> passed to
|
|
@@ -132,6 +156,7 @@ public class MultipleOutputs {
|
|
|
* Counters group used by the counters of MultipleOutputs.
|
|
|
*/
|
|
|
private static final String COUNTERS_GROUP = MultipleOutputs.class.getName();
|
|
|
+ private static final Logger LOG = LoggerFactory.getLogger(MultipleOutputs.class);
|
|
|
|
|
|
/**
|
|
|
* Checks if a named output is alreadyDefined or not.
|
|
@@ -381,6 +406,11 @@ public class MultipleOutputs {
|
|
|
private Map<String, RecordWriter> recordWriters;
|
|
|
private boolean countersEnabled;
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ synchronized void setRecordWriters(Map<String, RecordWriter> recordWriters) {
|
|
|
+ this.recordWriters = recordWriters;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Creates and initializes multiple named outputs support, it should be
|
|
|
* instantiated in the Mapper/Reducer configure method.
|
|
@@ -528,8 +558,41 @@ public class MultipleOutputs {
|
|
|
* could not be closed properly.
|
|
|
*/
|
|
|
public void close() throws IOException {
|
|
|
+ int nThreads = conf.getInt(MRConfig.MULTIPLE_OUTPUTS_CLOSE_THREAD_COUNT,
|
|
|
+ MRConfig.DEFAULT_MULTIPLE_OUTPUTS_CLOSE_THREAD_COUNT);
|
|
|
+ AtomicBoolean encounteredException = new AtomicBoolean(false);
|
|
|
+ ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat("MultipleOutputs-close")
|
|
|
+ .setUncaughtExceptionHandler(((t, e) -> {
|
|
|
+ LOG.error("Thread " + t + " failed unexpectedly", e);
|
|
|
+ encounteredException.set(true);
|
|
|
+ })).build();
|
|
|
+ ExecutorService executorService = Executors.newFixedThreadPool(nThreads, threadFactory);
|
|
|
+
|
|
|
+ List<Callable<Object>> callableList = new ArrayList<>(recordWriters.size());
|
|
|
+
|
|
|
for (RecordWriter writer : recordWriters.values()) {
|
|
|
- writer.close(null);
|
|
|
+ callableList.add(() -> {
|
|
|
+ try {
|
|
|
+ writer.close(null);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Error while closing MultipleOutput file", e);
|
|
|
+ encounteredException.set(true);
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ });
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ executorService.invokeAll(callableList);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ LOG.warn("Closing is Interrupted");
|
|
|
+ Thread.currentThread().interrupt();
|
|
|
+ } finally {
|
|
|
+ executorService.shutdown();
|
|
|
+ }
|
|
|
+
|
|
|
+ if (encounteredException.get()) {
|
|
|
+ throw new IOException(
|
|
|
+ "One or more threads encountered exception during close. See prior errors.");
|
|
|
}
|
|
|
}
|
|
|
|