YARN-2799. Cleanup TestLogAggregationService based on the change in YARN-90. Contributed by Zhihai Xu

This commit is contained in:
Junping Du 2015-02-20 09:43:39 -08:00
parent a64dd3d24b
commit c33ae271c2
2 changed files with 6 additions and 22 deletions

View File

@ -307,6 +307,9 @@ Release 2.7.0 - UNRELEASED
YARN-3076. Add API/Implementation to YarnClient to retrieve label-to-node
mapping. (Varun Saxena via wangda)
YARN-2799. Cleanup TestLogAggregationService based on the change in YARN-90.
(Zhihai Xu via junping_du)
OPTIMIZATIONS
YARN-2990. FairScheduler's delay-scheduling always waits for node-local and

View File

@ -248,7 +248,6 @@ public void testLocalFileDeletionAfterUpload() throws Exception {
};
checkEvents(appEventHandler, expectedEvents, true, "getType", "getApplicationID");
dispatcher.stop();
}
@Test
@ -295,7 +294,6 @@ public void testNoContainerOnNode() throws Exception {
ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED)
};
checkEvents(appEventHandler, expectedEvents, true, "getType", "getApplicationID");
dispatcher.stop();
logAggregationService.close();
}
@ -308,10 +306,7 @@ public void testMultipleAppsLogAggregation() throws Exception {
this.remoteRootLogDir.getAbsolutePath());
String[] fileNames = new String[] { "stdout", "stderr", "syslog" };
DrainDispatcher dispatcher = createDispatcher();
EventHandler<ApplicationEvent> appEventHandler = mock(EventHandler.class);
dispatcher.register(ApplicationEventType.class, appEventHandler);
LogAggregationService logAggregationService =
new LogAggregationService(dispatcher, this.context, this.delSrvc,
super.dirsHandler);
@ -441,7 +436,6 @@ public void testMultipleAppsLogAggregation() throws Exception {
ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED)
};
checkEvents(appEventHandler, expectedFinishedEvents, false, "getType", "getApplicationID");
dispatcher.stop();
}
@Test
@ -518,8 +512,7 @@ public void testVerifyAndCreateRemoteDirNonExistence()
File aNewFile = new File(String.valueOf("tmp"+System.currentTimeMillis()));
this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
aNewFile.getAbsolutePath());
DrainDispatcher dispatcher = createDispatcher();
LogAggregationService logAggregationService = spy(
new LogAggregationService(dispatcher, this.context, this.delSrvc,
super.dirsHandler));
@ -590,6 +583,7 @@ protected FileSystem getFileSystem(Configuration conf) {
verify(spyFs, never()).mkdirs(eq(appDir3), isA(FsPermission.class));
aggSvc.stop();
aggSvc.close();
dispatcher.stop();
}
@Test
@ -1122,10 +1116,6 @@ public void testLogAggregationServiceWithPatterns() throws Exception {
this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
this.remoteRootLogDir.getAbsolutePath());
DrainDispatcher dispatcher = createDispatcher();
EventHandler<ApplicationEvent> appEventHandler = mock(EventHandler.class);
dispatcher.register(ApplicationEventType.class, appEventHandler);
ApplicationId application1 = BuilderUtils.newApplicationId(1234, 1);
ApplicationId application2 = BuilderUtils.newApplicationId(1234, 2);
ApplicationId application3 = BuilderUtils.newApplicationId(1234, 3);
@ -1282,7 +1272,6 @@ public void testLogAggregationServiceWithPatterns() throws Exception {
ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED) };
checkEvents(appEventHandler, expectedFinishedEvents, false, "getType",
"getApplicationID");
dispatcher.stop();
}
@Test (timeout = 50000)
@ -1319,10 +1308,6 @@ private void testLogAggregationService(boolean retentionSizeLimitation)
// again in next cycle.
this.conf.setLong(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600);
DrainDispatcher dispatcher = createDispatcher();
EventHandler<ApplicationEvent> appEventHandler = mock(EventHandler.class);
dispatcher.register(ApplicationEventType.class, appEventHandler);
ApplicationId application = BuilderUtils.newApplicationId(123456, 1);
ApplicationAttemptId appAttemptId =
BuilderUtils.newApplicationAttemptId(application, 1);
@ -1426,7 +1411,6 @@ private void testLogAggregationService(boolean retentionSizeLimitation)
new ContainerId[] { container }, logFiles3, 3, true);
logAggregationService.stop();
assertEquals(0, logAggregationService.getNumAggregators());
dispatcher.stop();
}
@ -1436,8 +1420,6 @@ public void testAddNewTokenSentFromRMForLogAggregation() throws Exception {
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
"kerberos");
UserGroupInformation.setConfiguration(conf);
DrainDispatcher dispatcher = createDispatcher();
dispatcher.register(ApplicationEventType.class, appEventHandler);
ApplicationId application1 = BuilderUtils.newApplicationId(1234, 1);
Application mockApp = mock(Application.class);
@ -1484,7 +1466,6 @@ public Boolean get() {
}
}, 1000, 20000);
logAggregationService.stop();
dispatcher.stop();
}
private int numOfLogsAvailable(LogAggregationService logAggregationService,