|
@@ -1575,6 +1575,102 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
"getApplicationID");
|
|
|
}
|
|
|
|
|
|
+ @SuppressWarnings("resource")
|
|
|
+ @Test (timeout = 50000)
|
|
|
+ public void testLogAggregationServiceWithPatternsAndIntervals()
|
|
|
+ throws Exception {
|
|
|
+ LogAggregationContext logAggregationContext =
|
|
|
+ Records.newRecord(LogAggregationContext.class);
|
|
|
+ // set IncludePattern and RolledLogsIncludePattern.
|
|
|
+ // When the app is running, we only aggregate the log with
|
|
|
+ // the name stdout. After the app finishes, we only aggregate
|
|
|
+ // the log with the name std_final.
|
|
|
+ logAggregationContext.setRolledLogsIncludePattern("stdout");
|
|
|
+ logAggregationContext.setIncludePattern("std_final");
|
|
|
+ this.conf.set(
|
|
|
+ YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
|
|
|
+ //configure YarnConfiguration.NM_REMOTE_APP_LOG_DIR to
|
|
|
+ //have fully qualified path
|
|
|
+ this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
|
|
|
+ this.remoteRootLogDir.toURI().toString());
|
|
|
+ this.conf.setLong(
|
|
|
+ YarnConfiguration.NM_LOG_AGGREGATION_ROLL_MONITORING_INTERVAL_SECONDS,
|
|
|
+ 3600);
|
|
|
+
|
|
|
+ this.conf.setLong(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600);
|
|
|
+
|
|
|
+ ApplicationId application =
|
|
|
+ BuilderUtils.newApplicationId(System.currentTimeMillis(), 1);
|
|
|
+ ApplicationAttemptId appAttemptId =
|
|
|
+ BuilderUtils.newApplicationAttemptId(application, 1);
|
|
|
+ ContainerId container = createContainer(appAttemptId, 1,
|
|
|
+ ContainerType.APPLICATION_MASTER);
|
|
|
+
|
|
|
+ ConcurrentMap<ApplicationId, Application> maps =
|
|
|
+ this.context.getApplications();
|
|
|
+ Application app = mock(Application.class);
|
|
|
+ maps.put(application, app);
|
|
|
+ when(app.getContainers()).thenReturn(this.context.getContainers());
|
|
|
+
|
|
|
+ LogAggregationService logAggregationService =
|
|
|
+ new LogAggregationService(dispatcher, context, this.delSrvc,
|
|
|
+ super.dirsHandler);
|
|
|
+
|
|
|
+ logAggregationService.init(this.conf);
|
|
|
+ logAggregationService.start();
|
|
|
+
|
|
|
+ // AppLogDir should be created
|
|
|
+ File appLogDir =
|
|
|
+ new File(localLogDir, ConverterUtils.toString(application));
|
|
|
+ appLogDir.mkdir();
|
|
|
+ logAggregationService.handle(new LogHandlerAppStartedEvent(application,
|
|
|
+ this.user, null, this.acls, logAggregationContext));
|
|
|
+
|
|
|
+ // Simulate log-file creation
|
|
|
+ // create std_final in log directory which will not be aggregated
|
|
|
+ // until the app finishes.
|
|
|
+ String[] logFilesWithFinalLog =
|
|
|
+ new String[] {"stdout", "std_final"};
|
|
|
+ writeContainerLogs(appLogDir, container, logFilesWithFinalLog);
|
|
|
+
|
|
|
+ // Do log aggregation
|
|
|
+ AppLogAggregatorImpl aggregator =
|
|
|
+ (AppLogAggregatorImpl) logAggregationService.getAppLogAggregators()
|
|
|
+ .get(application);
|
|
|
+
|
|
|
+ aggregator.doLogAggregationOutOfBand();
|
|
|
+
|
|
|
+ Assert.assertTrue(waitAndCheckLogNum(logAggregationService, application,
|
|
|
+ 50, 1, false, null));
|
|
|
+
|
|
|
+ String[] logFiles = new String[] { "stdout" };
|
|
|
+ verifyContainerLogs(logAggregationService, application,
|
|
|
+ new ContainerId[] {container}, logFiles, 1, true);
|
|
|
+
|
|
|
+ logAggregationService.handle(
|
|
|
+ new LogHandlerContainerFinishedEvent(container, 0));
|
|
|
+
|
|
|
+ dispatcher.await();
|
|
|
+
|
|
|
+ // Do the log aggregation after ContainerFinishedEvent but before
|
|
|
+ // AppFinishedEvent. The std_final is expected to be aggregated this time
|
|
|
+ // even if the app is running but the container finishes.
|
|
|
+ aggregator.doLogAggregationOutOfBand();
|
|
|
+
|
|
|
+ Assert.assertTrue(waitAndCheckLogNum(logAggregationService, application,
|
|
|
+ 50, 2, false, null));
|
|
|
+
|
|
|
+ // This container finishes.
|
|
|
+ // The log "std_final" should be aggregated this time.
|
|
|
+ String[] logFinalLog = new String[] {"std_final"};
|
|
|
+ verifyContainerLogs(logAggregationService, application,
|
|
|
+ new ContainerId[] {container}, logFinalLog, 1, true);
|
|
|
+
|
|
|
+ logAggregationService.handle(new LogHandlerAppFinishedEvent(application));
|
|
|
+
|
|
|
+ logAggregationService.stop();
|
|
|
+ }
|
|
|
+
|
|
|
@Test (timeout = 50000)
|
|
|
@SuppressWarnings("unchecked")
|
|
|
public void testNoneContainerPolicy() throws Exception {
|
|
@@ -1583,14 +1679,14 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
LogAggregationService logAggregationService = createLogAggregationService(
|
|
|
appId, NoneContainerLogAggregationPolicy.class, null);
|
|
|
|
|
|
- String[] logFiles = new String[] { "stdout" };
|
|
|
+ String[] logFiles = new String[] {"stdout"};
|
|
|
ContainerId container1 = finishContainer(appId, logAggregationService,
|
|
|
ContainerType.APPLICATION_MASTER, 1, 0, logFiles);
|
|
|
|
|
|
finishApplication(appId, logAggregationService);
|
|
|
|
|
|
verifyContainerLogs(logAggregationService, appId,
|
|
|
- new ContainerId[] { container1 }, logFiles, 0, false);
|
|
|
+ new ContainerId[] {container1}, logFiles, 0, false);
|
|
|
|
|
|
verifyLogAggFinishEvent(appId);
|
|
|
}
|