瀏覽代碼

YARN-4155. TestLogAggregationService.testLogAggregationServiceWithInterval failing. (Bibin A Chundatt via stevel)

Steve Loughran 9 年之前
父節點
當前提交
e2d59e2c7b

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -957,6 +957,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4000. RM crashes with NPE if leaf queue becomes parent queue during restart. 
     (Varun Saxena via jianhe)
 
+    YARN-4155. TestLogAggregationService.testLogAggregationServiceWithInterval failing
+    (Bibin A Chundatt via stevel)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java

@@ -22,6 +22,7 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -248,7 +249,7 @@ public abstract class BaseContainerManagerTest {
       public void delete(String user, Path subDir, Path... baseDirs) {
         // Don't do any deletions.
         LOG.info("Psuedo delete: user - " + user + ", subDir - " + subDir
-            + ", baseDirs - " + baseDirs); 
+            + ", baseDirs - " + Arrays.asList(baseDirs));
       };
     };
   }

+ 23 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java

@@ -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 {