|
@@ -23,12 +23,17 @@ import java.io.BufferedReader;
|
|
|
import java.io.DataOutputStream;
|
|
|
import java.io.File;
|
|
|
import java.io.FileInputStream;
|
|
|
+import java.io.Flushable;
|
|
|
import java.io.IOException;
|
|
|
import java.io.InputStream;
|
|
|
import java.io.InputStreamReader;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Enumeration;
|
|
|
import java.util.List;
|
|
|
+import java.util.concurrent.Executors;
|
|
|
+import java.util.concurrent.ScheduledExecutorService;
|
|
|
+import java.util.concurrent.ThreadFactory;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -44,6 +49,8 @@ import org.apache.hadoop.io.SecureIOUtils;
|
|
|
import org.apache.hadoop.mapreduce.JobID;
|
|
|
import org.apache.hadoop.mapreduce.util.ProcessTree;
|
|
|
import org.apache.hadoop.util.Shell;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
+import org.apache.hadoop.util.ShutdownHookManager;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.log4j.Appender;
|
|
|
import org.apache.log4j.LogManager;
|
|
@@ -262,7 +269,86 @@ public class TaskLog {
|
|
|
}
|
|
|
writeToIndexFile(logLocation, isCleanup);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ public static synchronized void syncLogsShutdown(
|
|
|
+ ScheduledExecutorService scheduler)
|
|
|
+ {
|
|
|
+ // flush standard streams
|
|
|
+ //
|
|
|
+ System.out.flush();
|
|
|
+ System.err.flush();
|
|
|
+
|
|
|
+ if (scheduler != null) {
|
|
|
+ scheduler.shutdownNow();
|
|
|
+ }
|
|
|
+
|
|
|
+ // flush & close all appenders
|
|
|
+ LogManager.shutdown();
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ public static synchronized void syncLogs() {
|
|
|
+ // flush standard streams
|
|
|
+ //
|
|
|
+ System.out.flush();
|
|
|
+ System.err.flush();
|
|
|
+
|
|
|
+ // flush flushable appenders
|
|
|
+ //
|
|
|
+ final Logger rootLogger = Logger.getRootLogger();
|
|
|
+ flushAppenders(rootLogger);
|
|
|
+ final Enumeration<Logger> allLoggers = rootLogger.getLoggerRepository().
|
|
|
+ getCurrentLoggers();
|
|
|
+ while (allLoggers.hasMoreElements()) {
|
|
|
+ final Logger l = allLoggers.nextElement();
|
|
|
+ flushAppenders(l);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ private static void flushAppenders(Logger l) {
|
|
|
+ final Enumeration<Appender> allAppenders = l.getAllAppenders();
|
|
|
+ while (allAppenders.hasMoreElements()) {
|
|
|
+ final Appender a = allAppenders.nextElement();
|
|
|
+ if (a instanceof Flushable) {
|
|
|
+ try {
|
|
|
+ ((Flushable) a).flush();
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ System.err.println(a + ": Failed to flush!"
|
|
|
+ + StringUtils.stringifyException(ioe));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public static ScheduledExecutorService createLogSyncer() {
|
|
|
+ final ScheduledExecutorService scheduler =
|
|
|
+ Executors.newSingleThreadScheduledExecutor(
|
|
|
+ new ThreadFactory() {
|
|
|
+ @Override
|
|
|
+ public Thread newThread(Runnable r) {
|
|
|
+ final Thread t = Executors.defaultThreadFactory().newThread(r);
|
|
|
+ t.setDaemon(true);
|
|
|
+ t.setName("Thread for syncLogs");
|
|
|
+ return t;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ ShutdownHookManager.get().addShutdownHook(new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ TaskLog.syncLogsShutdown(scheduler);
|
|
|
+ }
|
|
|
+ }, 50);
|
|
|
+ scheduler.scheduleWithFixedDelay(
|
|
|
+ new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ TaskLog.syncLogs();
|
|
|
+ }
|
|
|
+ }, 0L, 5L, TimeUnit.SECONDS);
|
|
|
+ return scheduler;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* The filter for userlogs.
|
|
|
*/
|