|
@@ -788,7 +788,9 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
// ContainerLogDir should be created
|
|
|
String containerStr = ConverterUtils.toString(containerId);
|
|
|
File containerLogDir = new File(appLogDir, containerStr);
|
|
|
- containerLogDir.mkdir();
|
|
|
+ boolean created = containerLogDir.mkdirs();
|
|
|
+ LOG.info("Created Dir:" + containerLogDir.getAbsolutePath() + " status :"
|
|
|
+ + created);
|
|
|
for (String fileType : fileName) {
|
|
|
Writer writer11 = new FileWriter(new File(containerLogDir, fileType));
|
|
|
writer11.write(containerStr + " Hello " + fileType + "!");
|
|
@@ -1926,7 +1928,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
//configure YarnConfiguration.NM_REMOTE_APP_LOG_DIR to
|
|
|
//have fully qualified path
|
|
|
this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
|
|
|
- "file://" + this.remoteRootLogDir.getAbsolutePath());
|
|
|
+ this.remoteRootLogDir.toURI().toString());
|
|
|
this.conf.setLong(
|
|
|
YarnConfiguration.NM_LOG_AGGREGATION_ROLL_MONITORING_INTERVAL_SECONDS,
|
|
|
3600);
|
|
@@ -1943,7 +1945,8 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
// again in next cycle.
|
|
|
this.conf.setLong(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600);
|
|
|
|
|
|
- ApplicationId application = BuilderUtils.newApplicationId(123456, 1);
|
|
|
+ ApplicationId application =
|
|
|
+ BuilderUtils.newApplicationId(System.currentTimeMillis(), 1);
|
|
|
ApplicationAttemptId appAttemptId =
|
|
|
BuilderUtils.newApplicationAttemptId(application, 1);
|
|
|
ContainerId container = createContainer(appAttemptId, 1,
|
|
@@ -2009,8 +2012,10 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
|
|
|
// Same logs will not be aggregated again.
|
|
|
// Only one aggregated log file in Remote file directory.
|
|
|
- Assert.assertEquals(numOfLogsAvailable(logAggregationService,
|
|
|
- application, true, null), 1);
|
|
|
+ Assert.assertTrue(
|
|
|
+ "Only one aggregated log file in Remote file directory expected",
|
|
|
+ waitAndCheckLogNum(logAggregationService, application, 50, 1, true,
|
|
|
+ null));
|
|
|
|
|
|
Thread.sleep(2000);
|
|
|
|
|
@@ -2134,6 +2139,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
FileContext.getFileContext(qualifiedLogDir.toUri(), this.conf)
|
|
|
.listStatus(appLogDir);
|
|
|
} catch (FileNotFoundException fnf) {
|
|
|
+ LOG.info("Context file not vailable: " + fnf);
|
|
|
return -1;
|
|
|
}
|
|
|
int count = 0;
|
|
@@ -2143,13 +2149,17 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
if (filename.contains(LogAggregationUtils.TMP_FILE_SUFFIX)
|
|
|
|| (lastLogFile != null && filename.contains(lastLogFile)
|
|
|
&& sizeLimited)) {
|
|
|
+ LOG.info("fileName :" + filename);
|
|
|
+ LOG.info("lastLogFile :" + lastLogFile);
|
|
|
return -1;
|
|
|
}
|
|
|
if (filename.contains(LogAggregationUtils
|
|
|
.getNodeString(logAggregationService.getNodeId()))) {
|
|
|
+ LOG.info("Node list filename :" + filename);
|
|
|
count++;
|
|
|
}
|
|
|
}
|
|
|
+ LOG.info("File Count :" + count);
|
|
|
return count;
|
|
|
}
|
|
|
|
|
@@ -2158,13 +2168,17 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
int maxAttempts, int expectNum, boolean sizeLimited, String lastLogFile)
|
|
|
throws IOException, InterruptedException {
|
|
|
int count = 0;
|
|
|
- while (numOfLogsAvailable(logAggregationService, application, sizeLimited,
|
|
|
- lastLogFile) != expectNum && count <= maxAttempts) {
|
|
|
+ int logFiles=numOfLogsAvailable(logAggregationService, application, sizeLimited,
|
|
|
+ lastLogFile);
|
|
|
+ while ((logFiles != expectNum)
|
|
|
+ && (count <= maxAttempts)) {
|
|
|
Thread.sleep(500);
|
|
|
count++;
|
|
|
+ logFiles =
|
|
|
+ numOfLogsAvailable(logAggregationService, application, sizeLimited,
|
|
|
+ lastLogFile);
|
|
|
}
|
|
|
- return numOfLogsAvailable(logAggregationService, application, sizeLimited,
|
|
|
- lastLogFile) == expectNum;
|
|
|
+ return (logFiles == expectNum);
|
|
|
}
|
|
|
|
|
|
private static class LogFileStatusInLastCycle {
|