|
@@ -19,11 +19,7 @@
|
|
|
package org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
-import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.Arrays;
|
|
|
-import java.util.Collections;
|
|
|
-import java.util.Comparator;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.List;
|
|
@@ -38,8 +34,6 @@ import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileContext;
|
|
|
-import org.apache.hadoop.fs.FileStatus;
|
|
|
-import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
@@ -57,7 +51,9 @@ import org.apache.hadoop.yarn.event.Dispatcher;
|
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat;
|
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
|
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
|
|
|
-import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogWriter;
|
|
|
+import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileController;
|
|
|
+import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext;
|
|
|
+import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationTFileController;
|
|
|
import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
|
|
|
import org.apache.hadoop.yarn.server.api.ContainerLogAggregationPolicy;
|
|
|
import org.apache.hadoop.yarn.server.api.ContainerLogContext;
|
|
@@ -71,7 +67,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.deletion.task.DeletionTask;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.deletion.task.FileDeletionTask;
|
|
|
-import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
|
import org.apache.hadoop.yarn.util.Times;
|
|
|
|
|
@@ -86,18 +81,6 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(AppLogAggregatorImpl.class);
|
|
|
private static final int THREAD_SLEEP_TIME = 1000;
|
|
|
- // This is temporary solution. The configuration will be deleted once
|
|
|
- // we find a more scalable method to only write a single log file per LRS.
|
|
|
- private static final String NM_LOG_AGGREGATION_NUM_LOG_FILES_SIZE_PER_APP
|
|
|
- = YarnConfiguration.NM_PREFIX + "log-aggregation.num-log-files-per-app";
|
|
|
- private static final int
|
|
|
- DEFAULT_NM_LOG_AGGREGATION_NUM_LOG_FILES_SIZE_PER_APP = 30;
|
|
|
-
|
|
|
- // This configuration is for debug and test purpose. By setting
|
|
|
- // this configuration as true. We can break the lower bound of
|
|
|
- // NM_LOG_AGGREGATION_ROLL_MONITORING_INTERVAL_SECONDS.
|
|
|
- private static final String NM_LOG_AGGREGATION_DEBUG_ENABLED
|
|
|
- = YarnConfiguration.NM_PREFIX + "log-aggregation.debug-enabled";
|
|
|
|
|
|
private final LocalDirsHandlerService dirsHandler;
|
|
|
private final Dispatcher dispatcher;
|
|
@@ -118,10 +101,8 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
private final FileContext lfs;
|
|
|
private final LogAggregationContext logAggregationContext;
|
|
|
private final Context context;
|
|
|
- private final int retentionSize;
|
|
|
- private final long rollingMonitorInterval;
|
|
|
- private final boolean logAggregationInRolling;
|
|
|
private final NodeId nodeId;
|
|
|
+ private final LogAggregationFileControllerContext logControllerContext;
|
|
|
|
|
|
// These variables are only for testing
|
|
|
private final AtomicBoolean waiting = new AtomicBoolean(false);
|
|
@@ -134,6 +115,8 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
new HashMap<ContainerId, ContainerLogAggregator>();
|
|
|
private final ContainerLogAggregationPolicy logAggPolicy;
|
|
|
|
|
|
+ private final LogAggregationFileController logAggregationFileController;
|
|
|
+
|
|
|
|
|
|
/**
|
|
|
* The value recovered from state store to determine the age of application
|
|
@@ -151,7 +134,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
FileContext lfs, long rollingMonitorInterval) {
|
|
|
this(dispatcher, deletionService, conf, appId, userUgi, nodeId,
|
|
|
dirsHandler, remoteNodeLogFileForApp, appAcls,
|
|
|
- logAggregationContext, context, lfs, rollingMonitorInterval, -1);
|
|
|
+ logAggregationContext, context, lfs, rollingMonitorInterval, -1, null);
|
|
|
}
|
|
|
|
|
|
public AppLogAggregatorImpl(Dispatcher dispatcher,
|
|
@@ -162,6 +145,21 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
LogAggregationContext logAggregationContext, Context context,
|
|
|
FileContext lfs, long rollingMonitorInterval,
|
|
|
long recoveredLogInitedTime) {
|
|
|
+ this(dispatcher, deletionService, conf, appId, userUgi, nodeId,
|
|
|
+ dirsHandler, remoteNodeLogFileForApp, appAcls,
|
|
|
+ logAggregationContext, context, lfs, rollingMonitorInterval,
|
|
|
+ recoveredLogInitedTime, null);
|
|
|
+ }
|
|
|
+
|
|
|
+ public AppLogAggregatorImpl(Dispatcher dispatcher,
|
|
|
+ DeletionService deletionService, Configuration conf,
|
|
|
+ ApplicationId appId, UserGroupInformation userUgi, NodeId nodeId,
|
|
|
+ LocalDirsHandlerService dirsHandler, Path remoteNodeLogFileForApp,
|
|
|
+ Map<ApplicationAccessType, String> appAcls,
|
|
|
+ LogAggregationContext logAggregationContext, Context context,
|
|
|
+ FileContext lfs, long rollingMonitorInterval,
|
|
|
+ long recoveredLogInitedTime,
|
|
|
+ LogAggregationFileController logAggregationFileController) {
|
|
|
this.dispatcher = dispatcher;
|
|
|
this.conf = conf;
|
|
|
this.delService = deletionService;
|
|
@@ -169,31 +167,41 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
this.applicationId = appId.toString();
|
|
|
this.userUgi = userUgi;
|
|
|
this.dirsHandler = dirsHandler;
|
|
|
- this.remoteNodeLogFileForApp = remoteNodeLogFileForApp;
|
|
|
- this.remoteNodeTmpLogFileForApp = getRemoteNodeTmpLogFileForApp();
|
|
|
this.pendingContainers = new LinkedBlockingQueue<ContainerId>();
|
|
|
this.appAcls = appAcls;
|
|
|
this.lfs = lfs;
|
|
|
this.logAggregationContext = logAggregationContext;
|
|
|
this.context = context;
|
|
|
this.nodeId = nodeId;
|
|
|
- int configuredRentionSize =
|
|
|
- conf.getInt(NM_LOG_AGGREGATION_NUM_LOG_FILES_SIZE_PER_APP,
|
|
|
- DEFAULT_NM_LOG_AGGREGATION_NUM_LOG_FILES_SIZE_PER_APP);
|
|
|
- if (configuredRentionSize <= 0) {
|
|
|
- this.retentionSize =
|
|
|
- DEFAULT_NM_LOG_AGGREGATION_NUM_LOG_FILES_SIZE_PER_APP;
|
|
|
+ this.logAggPolicy = getLogAggPolicy(conf);
|
|
|
+ this.recoveredLogInitedTime = recoveredLogInitedTime;
|
|
|
+ if (logAggregationFileController == null) {
|
|
|
+ // by default, use T-File Controller
|
|
|
+ this.logAggregationFileController = new LogAggregationTFileController();
|
|
|
+ this.logAggregationFileController.initialize(conf, "TFile");
|
|
|
+ this.logAggregationFileController.verifyAndCreateRemoteLogDir();
|
|
|
+ this.logAggregationFileController.createAppDir(
|
|
|
+ this.userUgi.getShortUserName(), appId, userUgi);
|
|
|
+ this.remoteNodeLogFileForApp = this.logAggregationFileController
|
|
|
+ .getRemoteNodeLogFileForApp(appId,
|
|
|
+ this.userUgi.getShortUserName(), nodeId);
|
|
|
+ this.remoteNodeTmpLogFileForApp = getRemoteNodeTmpLogFileForApp();
|
|
|
} else {
|
|
|
- this.retentionSize = configuredRentionSize;
|
|
|
+ this.logAggregationFileController = logAggregationFileController;
|
|
|
+ this.remoteNodeLogFileForApp = remoteNodeLogFileForApp;
|
|
|
+ this.remoteNodeTmpLogFileForApp = getRemoteNodeTmpLogFileForApp();
|
|
|
}
|
|
|
- this.rollingMonitorInterval = rollingMonitorInterval;
|
|
|
- this.logAggregationInRolling =
|
|
|
- this.rollingMonitorInterval <= 0 || this.logAggregationContext == null
|
|
|
+ boolean logAggregationInRolling =
|
|
|
+ rollingMonitorInterval <= 0 || this.logAggregationContext == null
|
|
|
|| this.logAggregationContext.getRolledLogsIncludePattern() == null
|
|
|
|| this.logAggregationContext.getRolledLogsIncludePattern()
|
|
|
- .isEmpty() ? false : true;
|
|
|
- this.logAggPolicy = getLogAggPolicy(conf);
|
|
|
- this.recoveredLogInitedTime = recoveredLogInitedTime;
|
|
|
+ .isEmpty() ? false : true;
|
|
|
+ logControllerContext = new LogAggregationFileControllerContext(
|
|
|
+ this.remoteNodeLogFileForApp,
|
|
|
+ this.remoteNodeTmpLogFileForApp,
|
|
|
+ logAggregationInRolling,
|
|
|
+ rollingMonitorInterval,
|
|
|
+ this.appId, this.appAcls, this.nodeId, this.userUgi);
|
|
|
}
|
|
|
|
|
|
private ContainerLogAggregationPolicy getLogAggPolicy(Configuration conf) {
|
|
@@ -293,14 +301,9 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
logAggregationTimes++;
|
|
|
String diagnosticMessage = "";
|
|
|
boolean logAggregationSucceedInThisCycle = true;
|
|
|
- try (LogWriter writer = createLogWriter()) {
|
|
|
+ try {
|
|
|
try {
|
|
|
- writer.initialize(this.conf, this.remoteNodeTmpLogFileForApp,
|
|
|
- this.userUgi);
|
|
|
- // Write ACLs once when the writer is created.
|
|
|
- writer.writeApplicationACLs(appAcls);
|
|
|
- writer.writeApplicationOwner(this.userUgi.getShortUserName());
|
|
|
-
|
|
|
+ logAggregationFileController.initializeWriter(logControllerContext);
|
|
|
} catch (IOException e1) {
|
|
|
logAggregationSucceedInThisCycle = false;
|
|
|
LOG.error("Cannot create writer for app " + this.applicationId
|
|
@@ -318,8 +321,8 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
containerLogAggregators.put(container, aggregator);
|
|
|
}
|
|
|
Set<Path> uploadedFilePathsInThisCycle =
|
|
|
- aggregator.doContainerLogAggregation(writer, appFinished,
|
|
|
- finishedContainers.contains(container));
|
|
|
+ aggregator.doContainerLogAggregation(logAggregationFileController,
|
|
|
+ appFinished, finishedContainers.contains(container));
|
|
|
if (uploadedFilePathsInThisCycle.size() > 0) {
|
|
|
uploadedLogsInThisCycle = true;
|
|
|
List<Path> uploadedFilePathsInThisCycleList = new ArrayList<>();
|
|
@@ -337,60 +340,28 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // Before upload logs, make sure the number of existing logs
|
|
|
- // is smaller than the configured NM log aggregation retention size.
|
|
|
- if (uploadedLogsInThisCycle && logAggregationInRolling) {
|
|
|
- cleanOldLogs();
|
|
|
- cleanupOldLogTimes++;
|
|
|
- }
|
|
|
-
|
|
|
- long currentTime = System.currentTimeMillis();
|
|
|
- final Path renamedPath = getRenamedPath(currentTime);
|
|
|
-
|
|
|
- final boolean rename = uploadedLogsInThisCycle;
|
|
|
+ logControllerContext.setUploadedLogsInThisCycle(uploadedLogsInThisCycle);
|
|
|
+ logControllerContext.setLogUploadTimeStamp(System.currentTimeMillis());
|
|
|
+ logControllerContext.increLogAggregationTimes();
|
|
|
try {
|
|
|
- userUgi.doAs(new PrivilegedExceptionAction<Object>() {
|
|
|
- @Override
|
|
|
- public Object run() throws Exception {
|
|
|
- FileSystem remoteFS = remoteNodeLogFileForApp.getFileSystem(conf);
|
|
|
- if (rename) {
|
|
|
- remoteFS.rename(remoteNodeTmpLogFileForApp, renamedPath);
|
|
|
- } else {
|
|
|
- remoteFS.delete(remoteNodeTmpLogFileForApp, false);
|
|
|
- }
|
|
|
- return null;
|
|
|
- }
|
|
|
- });
|
|
|
- diagnosticMessage =
|
|
|
- "Log uploaded successfully for Application: " + appId
|
|
|
- + " in NodeManager: "
|
|
|
- + LogAggregationUtils.getNodeString(nodeId) + " at "
|
|
|
- + Times.format(currentTime) + "\n";
|
|
|
+ this.logAggregationFileController.postWrite(logControllerContext);
|
|
|
+ diagnosticMessage = "Log uploaded successfully for Application: "
|
|
|
+ + appId + " in NodeManager: "
|
|
|
+ + LogAggregationUtils.getNodeString(nodeId) + " at "
|
|
|
+ + Times.format(logControllerContext.getLogUploadTimeStamp())
|
|
|
+ + "\n";
|
|
|
} catch (Exception e) {
|
|
|
- LOG.error(
|
|
|
- "Failed to move temporary log file to final location: ["
|
|
|
- + remoteNodeTmpLogFileForApp + "] to ["
|
|
|
- + renamedPath + "]", e);
|
|
|
- diagnosticMessage =
|
|
|
- "Log uploaded failed for Application: " + appId
|
|
|
- + " in NodeManager: "
|
|
|
- + LogAggregationUtils.getNodeString(nodeId) + " at "
|
|
|
- + Times.format(currentTime) + "\n";
|
|
|
+ diagnosticMessage = e.getMessage();
|
|
|
renameTemporaryLogFileFailed = true;
|
|
|
logAggregationSucceedInThisCycle = false;
|
|
|
}
|
|
|
} finally {
|
|
|
sendLogAggregationReport(logAggregationSucceedInThisCycle,
|
|
|
diagnosticMessage, appFinished);
|
|
|
+ logAggregationFileController.closeWriter();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private Path getRenamedPath(long currentTime) {
|
|
|
- return this.rollingMonitorInterval <= 0 ? remoteNodeLogFileForApp
|
|
|
- : new Path(remoteNodeLogFileForApp.getParent(),
|
|
|
- remoteNodeLogFileForApp.getName() + "_" + currentTime);
|
|
|
- }
|
|
|
-
|
|
|
private void addCredentials() {
|
|
|
if (UserGroupInformation.isSecurityEnabled()) {
|
|
|
Credentials systemCredentials =
|
|
@@ -407,11 +378,6 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
- protected LogWriter createLogWriter() {
|
|
|
- return new LogWriter();
|
|
|
- }
|
|
|
-
|
|
|
private void sendLogAggregationReport(
|
|
|
boolean logAggregationSucceedInThisCycle, String diagnosticMessage,
|
|
|
boolean appFinished) {
|
|
@@ -442,60 +408,6 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
this.context.getLogAggregationStatusForApps().add(report);
|
|
|
}
|
|
|
|
|
|
- private void cleanOldLogs() {
|
|
|
- try {
|
|
|
- final FileSystem remoteFS =
|
|
|
- this.remoteNodeLogFileForApp.getFileSystem(conf);
|
|
|
- Path appDir =
|
|
|
- this.remoteNodeLogFileForApp.getParent().makeQualified(
|
|
|
- remoteFS.getUri(), remoteFS.getWorkingDirectory());
|
|
|
- Set<FileStatus> status =
|
|
|
- new HashSet<FileStatus>(Arrays.asList(remoteFS.listStatus(appDir)));
|
|
|
-
|
|
|
- Iterable<FileStatus> mask =
|
|
|
- Iterables.filter(status, new Predicate<FileStatus>() {
|
|
|
- @Override
|
|
|
- public boolean apply(FileStatus next) {
|
|
|
- return next.getPath().getName()
|
|
|
- .contains(LogAggregationUtils.getNodeString(nodeId))
|
|
|
- && !next.getPath().getName().endsWith(
|
|
|
- LogAggregationUtils.TMP_FILE_SUFFIX);
|
|
|
- }
|
|
|
- });
|
|
|
- status = Sets.newHashSet(mask);
|
|
|
- // Normally, we just need to delete one oldest log
|
|
|
- // before we upload a new log.
|
|
|
- // If we can not delete the older logs in this cycle,
|
|
|
- // we will delete them in next cycle.
|
|
|
- if (status.size() >= this.retentionSize) {
|
|
|
- // sort by the lastModificationTime ascending
|
|
|
- List<FileStatus> statusList = new ArrayList<FileStatus>(status);
|
|
|
- Collections.sort(statusList, new Comparator<FileStatus>() {
|
|
|
- public int compare(FileStatus s1, FileStatus s2) {
|
|
|
- return s1.getModificationTime() < s2.getModificationTime() ? -1
|
|
|
- : s1.getModificationTime() > s2.getModificationTime() ? 1 : 0;
|
|
|
- }
|
|
|
- });
|
|
|
- for (int i = 0 ; i <= statusList.size() - this.retentionSize; i++) {
|
|
|
- final FileStatus remove = statusList.get(i);
|
|
|
- try {
|
|
|
- userUgi.doAs(new PrivilegedExceptionAction<Object>() {
|
|
|
- @Override
|
|
|
- public Object run() throws Exception {
|
|
|
- remoteFS.delete(remove.getPath(), false);
|
|
|
- return null;
|
|
|
- }
|
|
|
- });
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.error("Failed to delete " + remove.getPath(), e);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.error("Failed to clean old logs", e);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
@SuppressWarnings("unchecked")
|
|
|
@Override
|
|
|
public void run() {
|
|
@@ -523,8 +435,8 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
synchronized(this) {
|
|
|
try {
|
|
|
waiting.set(true);
|
|
|
- if (logAggregationInRolling) {
|
|
|
- wait(this.rollingMonitorInterval * 1000);
|
|
|
+ if (logControllerContext.isLogAggregationInRolling()) {
|
|
|
+ wait(logControllerContext.getRollingMonitorInterval() * 1000);
|
|
|
if (this.appFinishing.get() || this.aborted.get()) {
|
|
|
break;
|
|
|
}
|
|
@@ -653,7 +565,8 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
recoveredLogInitedTime, logRetentionSecs * 1000);
|
|
|
}
|
|
|
|
|
|
- public Set<Path> doContainerLogAggregation(LogWriter writer,
|
|
|
+ public Set<Path> doContainerLogAggregation(
|
|
|
+ LogAggregationFileController logAggregationFileController,
|
|
|
boolean appFinished, boolean containerFinished) {
|
|
|
LOG.info("Uploading logs for container " + containerId
|
|
|
+ ". Current good log dirs are "
|
|
@@ -665,7 +578,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
this.uploadedFileMeta, retentionContext, appFinished,
|
|
|
containerFinished);
|
|
|
try {
|
|
|
- writer.append(logKey, logValue);
|
|
|
+ logAggregationFileController.write(logKey, logValue);
|
|
|
} catch (Exception e) {
|
|
|
LOG.error("Couldn't upload logs for " + containerId
|
|
|
+ ". Skipping this container.", e);
|
|
@@ -708,4 +621,15 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
|
|
|
int getCleanupOldLogTimes() {
|
|
|
return this.cleanupOldLogTimes;
|
|
|
}
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public LogAggregationFileController getLogAggregationFileController() {
|
|
|
+ return this.logAggregationFileController;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public LogAggregationFileControllerContext
|
|
|
+ getLogAggregationFileControllerContext() {
|
|
|
+ return this.logControllerContext;
|
|
|
+ }
|
|
|
}
|