|
@@ -20,10 +20,14 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregatio
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.Map;
|
|
|
+import java.util.Set;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.ConcurrentMap;
|
|
|
import java.util.concurrent.ExecutorService;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
+
|
|
|
+import org.apache.hadoop.security.token.SecretManager;
|
|
|
+
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
@@ -58,6 +62,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.eve
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerContainerFinishedEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEvent;
|
|
|
|
|
|
+
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
|
|
@@ -83,6 +88,9 @@ public class LogAggregationService extends AbstractService implements
|
|
|
|
|
|
private final ConcurrentMap<ApplicationId, AppLogAggregator> appLogAggregators;
|
|
|
|
|
|
+ // Holds applications whose aggregation is disable due to invalid Token
|
|
|
+ private final Set<ApplicationId> invalidTokenApps;
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
ExecutorService threadPool;
|
|
|
|
|
@@ -95,6 +103,7 @@ public class LogAggregationService extends AbstractService implements
|
|
|
this.dirsHandler = dirsHandler;
|
|
|
this.appLogAggregators =
|
|
|
new ConcurrentHashMap<ApplicationId, AppLogAggregator>();
|
|
|
+ this.invalidTokenApps = ConcurrentHashMap.newKeySet();
|
|
|
}
|
|
|
|
|
|
protected void serviceInit(Configuration conf) throws Exception {
|
|
@@ -224,8 +233,8 @@ public class LogAggregationService extends AbstractService implements
|
|
|
userUgi.addCredentials(credentials);
|
|
|
}
|
|
|
|
|
|
- LogAggregationFileController logAggregationFileController
|
|
|
- = getLogAggregationFileController(getConfig());
|
|
|
+ LogAggregationFileController logAggregationFileController =
|
|
|
+ getLogAggregationFileController(getConfig());
|
|
|
logAggregationFileController.verifyAndCreateRemoteLogDir();
|
|
|
// New application
|
|
|
final AppLogAggregator appLogAggregator =
|
|
@@ -245,14 +254,16 @@ public class LogAggregationService extends AbstractService implements
|
|
|
logAggregationFileController.createAppDir(user, appId, userUgi);
|
|
|
} catch (Exception e) {
|
|
|
appLogAggregator.disableLogAggregation();
|
|
|
+
|
|
|
+ // add to disabled aggregators if due to InvalidToken
|
|
|
+ if (e.getCause() instanceof SecretManager.InvalidToken) {
|
|
|
+ invalidTokenApps.add(appId);
|
|
|
+ }
|
|
|
if (!(e instanceof YarnRuntimeException)) {
|
|
|
appDirException = new YarnRuntimeException(e);
|
|
|
} else {
|
|
|
appDirException = (YarnRuntimeException)e;
|
|
|
}
|
|
|
- appLogAggregators.remove(appId);
|
|
|
- closeFileSystems(userUgi);
|
|
|
- throw appDirException;
|
|
|
}
|
|
|
|
|
|
// TODO Get the user configuration for the list of containers that need log
|
|
@@ -270,6 +281,10 @@ public class LogAggregationService extends AbstractService implements
|
|
|
}
|
|
|
};
|
|
|
this.threadPool.execute(aggregatorWrapper);
|
|
|
+
|
|
|
+ if (appDirException != null) {
|
|
|
+ throw appDirException;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
protected void closeFileSystems(final UserGroupInformation userUgi) {
|
|
@@ -307,17 +322,20 @@ public class LogAggregationService extends AbstractService implements
|
|
|
|
|
|
// App is complete. Finish up any containers' pending log aggregation and
|
|
|
// close the application specific logFile.
|
|
|
-
|
|
|
- AppLogAggregator aggregator = this.appLogAggregators.get(appId);
|
|
|
- if (aggregator == null) {
|
|
|
- LOG.warn("Log aggregation is not initialized for " + appId
|
|
|
- + ", did it fail to start?");
|
|
|
- this.dispatcher.getEventHandler().handle(
|
|
|
- new ApplicationEvent(appId,
|
|
|
- ApplicationEventType.APPLICATION_LOG_HANDLING_FAILED));
|
|
|
- return;
|
|
|
+ try {
|
|
|
+ AppLogAggregator aggregator = this.appLogAggregators.get(appId);
|
|
|
+ if (aggregator == null) {
|
|
|
+ LOG.warn("Log aggregation is not initialized for " + appId
|
|
|
+ + ", did it fail to start?");
|
|
|
+ this.dispatcher.getEventHandler().handle(new ApplicationEvent(appId,
|
|
|
+ ApplicationEventType.APPLICATION_LOG_HANDLING_FAILED));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ aggregator.finishLogAggregation();
|
|
|
+ } finally {
|
|
|
+ // Remove invalid Token Apps
|
|
|
+ invalidTokenApps.remove(appId);
|
|
|
}
|
|
|
- aggregator.finishLogAggregation();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -344,12 +362,47 @@ public class LogAggregationService extends AbstractService implements
|
|
|
(LogHandlerAppFinishedEvent) event;
|
|
|
stopApp(appFinishedEvent.getApplicationId());
|
|
|
break;
|
|
|
+ case LOG_AGG_TOKEN_UPDATE:
|
|
|
+ checkAndEnableAppAggregators();
|
|
|
+ break;
|
|
|
default:
|
|
|
; // Ignore
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
+ private void checkAndEnableAppAggregators() {
|
|
|
+ for (ApplicationId appId : invalidTokenApps) {
|
|
|
+ try {
|
|
|
+ AppLogAggregator aggregator = appLogAggregators.get(appId);
|
|
|
+ if (aggregator != null) {
|
|
|
+ Credentials credentials =
|
|
|
+ context.getSystemCredentialsForApps().get(appId);
|
|
|
+ if (credentials != null) {
|
|
|
+ // Create the app dir again with
|
|
|
+ LogAggregationFileController logAggregationFileController =
|
|
|
+ getLogAggregationFileController(getConfig());
|
|
|
+ UserGroupInformation userUgi =
|
|
|
+ aggregator.updateCredentials(credentials);
|
|
|
+ logAggregationFileController
|
|
|
+ .createAppDir(userUgi.getShortUserName(), appId, userUgi);
|
|
|
+ aggregator.enableLogAggregation();
|
|
|
+ }
|
|
|
+ invalidTokenApps.remove(appId);
|
|
|
+ LOG.info("LogAggregation enabled for application {}", appId);
|
|
|
+ }
|
|
|
+ } catch (Exception e) {
|
|
|
+ //Ignore exception
|
|
|
+ LOG.warn("Enable aggregators failed {}", appId);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Set<ApplicationId> getInvalidTokenApps() {
|
|
|
+ return invalidTokenApps;
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
public ConcurrentMap<ApplicationId, AppLogAggregator> getAppLogAggregators() {
|
|
|
return this.appLogAggregators;
|