You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by as...@apache.org on 2017/09/11 06:49:07 UTC

[42/50] [abbrv] hadoop git commit: YARN-7072. Add a new log aggregation file format controller. Contributed by Xuan Gong.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
index aaed538..989b326 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/LogAggregationTFileController.java
@@ -27,7 +27,6 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.math3.util.Pair;
@@ -192,7 +191,7 @@ public class LogAggregationTFileController
           while (valueStream != null) {
             if (getAllContainers || (key.toString().equals(containerIdStr))) {
               if (createPrintStream) {
-                os = createPrintStream(
+                os = LogToolUtils.createPrintStream(
                     logRequest.getOutputLocalDir(),
                     thisNodeFile.getPath().getName(), key.toString());
               }
@@ -209,12 +208,7 @@ public class LogAggregationTFileController
                           Times.format(thisNodeFile.getModificationTime()),
                           valueStream, os, buf,
                           ContainerLogAggregationType.AGGREGATED);
-                      StringBuilder sb = new StringBuilder();
-                      String endOfFile = "End of LogType:" + fileType;
-                      sb.append("\n" + endOfFile + "\n");
-                      sb.append(StringUtils.repeat("*", endOfFile.length() + 50)
-                          + "\n\n");
-                      byte[] b = sb.toString().getBytes(
+                      byte[] b = aggregatedLogSuffix(fileType).getBytes(
                           Charset.forName("UTF-8"));
                       os.write(b, 0, b.length);
                       findLogs = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java
new file mode 100644
index 0000000..5f61710
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/TestLogAggregationIndexFileController.java
@@ -0,0 +1,316 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.logaggregation.filecontroller.ifile;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.Writer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
+import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
+import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
+import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo;
+import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Function test for {@link LogAggregationIndexFileController}.
+ *
+ */
+public class TestLogAggregationIndexFileController {
+
+  private final String rootLocalLogDir = "target/LocalLogs";
+  private final Path rootLocalLogDirPath = new Path(rootLocalLogDir);
+  private final String remoteLogDir = "target/remote-app";
+  private static final FsPermission LOG_FILE_UMASK = FsPermission
+      .createImmutable((short) (0777));
+  private static final UserGroupInformation USER_UGI = UserGroupInformation
+      .createRemoteUser("testUser");
+  private FileSystem fs;
+  private Configuration conf;
+  private ApplicationId appId;
+  private ContainerId containerId;
+  private NodeId nodeId;
+
+  private ByteArrayOutputStream sysOutStream;
+  private PrintStream sysOut;
+
+  private ByteArrayOutputStream sysErrStream;
+  private PrintStream sysErr;
+
+  @Before
+  public void setUp() throws IOException {
+    appId = ApplicationId.newInstance(123456, 1);
+    ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
+        appId, 1);
+    containerId = ContainerId.newContainerId(attemptId, 1);
+    nodeId = NodeId.newInstance("localhost", 9999);
+    conf = new Configuration();
+    conf.set("yarn.log-aggregation.Indexed.remote-app-log-dir",
+        remoteLogDir);
+    conf.set("yarn.log-aggregation.Indexed.remote-app-log-dir-suffix",
+        "logs");
+    conf.set(YarnConfiguration.NM_LOG_AGG_COMPRESSION_TYPE, "gz");
+    fs = FileSystem.get(conf);
+    sysOutStream = new ByteArrayOutputStream();
+    sysOut =  new PrintStream(sysOutStream);
+    System.setOut(sysOut);
+
+    sysErrStream = new ByteArrayOutputStream();
+    sysErr = new PrintStream(sysErrStream);
+    System.setErr(sysErr);
+  }
+
+  @After
+  public void teardown() throws Exception {
+    fs.delete(rootLocalLogDirPath, true);
+    fs.delete(new Path(remoteLogDir), true);
+  }
+
+  @Test(timeout = 15000)
+  public void testLogAggregationIndexFileFormat() throws Exception {
+    if (fs.exists(rootLocalLogDirPath)) {
+      fs.delete(rootLocalLogDirPath, true);
+    }
+    assertTrue(fs.mkdirs(rootLocalLogDirPath));
+
+    Path appLogsDir = new Path(rootLocalLogDirPath, appId.toString());
+    if (fs.exists(appLogsDir)) {
+      fs.delete(appLogsDir, true);
+    }
+    assertTrue(fs.mkdirs(appLogsDir));
+
+    List<String> logTypes = new ArrayList<String>();
+    logTypes.add("syslog");
+    logTypes.add("stdout");
+    logTypes.add("stderr");
+
+    Set<File> files = new HashSet<>();
+
+    LogKey key1 = new LogKey(containerId.toString());
+
+    for(String logType : logTypes) {
+      File file = createAndWriteLocalLogFile(containerId, appLogsDir,
+          logType);
+      files.add(file);
+    }
+    LogValue value = mock(LogValue.class);
+    when(value.getPendingLogFilesToUploadForThisContainer()).thenReturn(files);
+
+    LogAggregationIndexedFileController fileFormat
+        = new LogAggregationIndexedFileController();
+    fileFormat.initialize(conf, "Indexed");
+
+    Map<ApplicationAccessType, String> appAcls = new HashMap<>();
+    Path appDir = fileFormat.getRemoteAppLogDir(appId,
+        USER_UGI.getShortUserName());
+    if (fs.exists(appDir)) {
+      fs.delete(appDir, true);
+    }
+    assertTrue(fs.mkdirs(appDir));
+
+    Path logPath = fileFormat.getRemoteNodeLogFileForApp(
+        appId, USER_UGI.getShortUserName(), nodeId);
+    LogAggregationFileControllerContext context =
+        new LogAggregationFileControllerContext(
+            logPath, logPath, true, 1000, appId, appAcls, nodeId, USER_UGI);
+    // initialize the writer
+    fileFormat.initializeWriter(context);
+
+    fileFormat.write(key1, value);
+    LogAggregationFileControllerContext record = mock(
+        LogAggregationFileControllerContext.class);
+    fileFormat.postWrite(record);
+    fileFormat.closeWriter();
+
+    ContainerLogsRequest logRequest = new ContainerLogsRequest();
+    logRequest.setAppId(appId);
+    logRequest.setNodeId(nodeId.toString());
+    logRequest.setAppOwner(USER_UGI.getShortUserName());
+    logRequest.setContainerId(containerId.toString());
+    logRequest.setBytes(Long.MAX_VALUE);
+    List<ContainerLogMeta> meta = fileFormat.readAggregatedLogsMeta(
+        logRequest);
+    Assert.assertTrue(meta.size() == 1);
+    List<String> fileNames = new ArrayList<>();
+    for (ContainerLogMeta log : meta) {
+      Assert.assertTrue(log.getContainerId().equals(containerId.toString()));
+      Assert.assertTrue(log.getNodeId().equals(nodeId.toString()));
+      Assert.assertTrue(log.getContainerLogMeta().size() == 3);
+      for (ContainerLogFileInfo file : log.getContainerLogMeta()) {
+        fileNames.add(file.getFileName());
+      }
+    }
+    fileNames.removeAll(logTypes);
+    Assert.assertTrue(fileNames.isEmpty());
+
+    boolean foundLogs = fileFormat.readAggregatedLogs(logRequest, System.out);
+    Assert.assertTrue(foundLogs);
+    for (String logType : logTypes) {
+      Assert.assertTrue(sysOutStream.toString().contains(logMessage(
+          containerId, logType)));
+    }
+    sysOutStream.reset();
+
+    // create a checksum file
+    Path checksumFile = new Path(fileFormat.getRemoteAppLogDir(
+        appId, USER_UGI.getShortUserName()),
+        LogAggregationUtils.getNodeString(nodeId)
+        + LogAggregationIndexedFileController.CHECK_SUM_FILE_SUFFIX);
+    FSDataOutputStream fInput = null;
+    try {
+      fInput = FileSystem.create(fs, checksumFile, LOG_FILE_UMASK);
+      fInput.writeLong(0);
+    } finally {
+      IOUtils.closeQuietly(fInput);
+    }
+    meta = fileFormat.readAggregatedLogsMeta(
+        logRequest);
+    Assert.assertTrue(meta.size() == 0);
+    foundLogs = fileFormat.readAggregatedLogs(logRequest, System.out);
+    Assert.assertFalse(foundLogs);
+    sysOutStream.reset();
+    fs.delete(checksumFile, false);
+    Assert.assertFalse(fs.exists(checksumFile));
+
+    List<String> newLogTypes = new ArrayList<>(logTypes);
+    files.clear();
+    newLogTypes.add("test1");
+    files.add(createAndWriteLocalLogFile(containerId, appLogsDir,
+        "test1"));
+    newLogTypes.add("test2");
+    files.add(createAndWriteLocalLogFile(containerId, appLogsDir,
+        "test2"));
+    LogValue value2 = mock(LogValue.class);
+    when(value2.getPendingLogFilesToUploadForThisContainer())
+        .thenReturn(files);
+
+    // initialize the writer
+    fileFormat.initializeWriter(context);
+    fileFormat.write(key1, value2);
+    fileFormat.closeWriter();
+
+    // We did not call postWriter which we would keep the checksum file.
+    // We can only get the logs/logmeta from the first write.
+    fileFormat.readAggregatedLogsMeta(
+        logRequest);
+    Assert.assertEquals(meta.size(), meta.size(), 1);
+    for (ContainerLogMeta log : meta) {
+      Assert.assertTrue(log.getContainerId().equals(containerId.toString()));
+      Assert.assertTrue(log.getNodeId().equals(nodeId.toString()));
+      Assert.assertTrue(log.getContainerLogMeta().size() == 3);
+      for (ContainerLogFileInfo file : log.getContainerLogMeta()) {
+        fileNames.add(file.getFileName());
+      }
+    }
+    fileNames.removeAll(logTypes);
+    Assert.assertTrue(fileNames.isEmpty());
+    foundLogs = fileFormat.readAggregatedLogs(logRequest, System.out);
+    Assert.assertTrue(foundLogs);
+    for (String logType : logTypes) {
+      Assert.assertTrue(sysOutStream.toString().contains(logMessage(
+          containerId, logType)));
+    }
+    Assert.assertFalse(sysOutStream.toString().contains(logMessage(
+        containerId, "test1")));
+    Assert.assertFalse(sysOutStream.toString().contains(logMessage(
+        containerId, "test2")));
+    sysOutStream.reset();
+
+    // Call postWrite and we should get all logs/logmetas for both
+    // first write and second write
+    fileFormat.initializeWriter(context);
+    fileFormat.write(key1, value2);
+    fileFormat.postWrite(record);
+    fileFormat.closeWriter();
+    fileFormat.readAggregatedLogsMeta(
+            logRequest);
+    Assert.assertEquals(meta.size(), meta.size(), 2);
+    for (ContainerLogMeta log : meta) {
+      Assert.assertTrue(log.getContainerId().equals(containerId.toString()));
+      Assert.assertTrue(log.getNodeId().equals(nodeId.toString()));
+      for (ContainerLogFileInfo file : log.getContainerLogMeta()) {
+        fileNames.add(file.getFileName());
+      }
+    }
+    fileNames.removeAll(newLogTypes);
+    Assert.assertTrue(fileNames.isEmpty());
+    foundLogs = fileFormat.readAggregatedLogs(logRequest, System.out);
+    Assert.assertTrue(foundLogs);
+    for (String logType : newLogTypes) {
+      Assert.assertTrue(sysOutStream.toString().contains(logMessage(
+          containerId, logType)));
+    }
+    sysOutStream.reset();
+  }
+
+  private File createAndWriteLocalLogFile(ContainerId containerId,
+      Path localLogDir, String logType) throws IOException {
+    File file = new File(localLogDir.toString(), logType);
+    if (file.exists()) {
+      file.delete();
+    }
+    file.createNewFile();
+    Writer writer = null;
+    try {
+      writer = new FileWriter(file);
+      writer.write(logMessage(containerId, logType));
+      writer.close();
+      return file;
+    } finally {
+      IOUtils.closeQuietly(writer);
+    }
+  }
+
+  private String logMessage(ContainerId containerId, String logType) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Hello " + containerId + " in " + logType + "!");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
index dc692a5..bca5d5c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
@@ -53,7 +53,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
-import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo;
 import org.apache.hadoop.yarn.logaggregation.TestContainerLogsUtils;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryClientService;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManagerOnTimelineStore;
@@ -851,7 +851,7 @@ public class TestAHSWebServices extends JerseyTestBase {
     for (ContainerLogsInfo logInfo : responseText) {
       if(logInfo.getLogType().equals(
           ContainerLogAggregationType.AGGREGATED.toString())) {
-        List<PerContainerLogFileInfo> logMeta = logInfo
+        List<ContainerLogFileInfo> logMeta = logInfo
             .getContainerLogsInfo();
         assertTrue(logMeta.size() == 1);
         assertEquals(logMeta.get(0).getFileName(), fileName);
@@ -879,7 +879,7 @@ public class TestAHSWebServices extends JerseyTestBase {
     for (ContainerLogsInfo logInfo : responseText) {
       if(logInfo.getLogType().equals(
           ContainerLogAggregationType.AGGREGATED.toString())) {
-        List<PerContainerLogFileInfo> logMeta = logInfo
+        List<ContainerLogFileInfo> logMeta = logInfo
             .getContainerLogsInfo();
         assertTrue(logMeta.size() == 1);
         assertEquals(logMeta.get(0).getFileName(), fileName);
@@ -917,7 +917,7 @@ public class TestAHSWebServices extends JerseyTestBase {
     assertTrue(responseText.size() == 1);
     assertEquals(responseText.get(0).getLogType(),
         ContainerLogAggregationType.AGGREGATED.toString());
-    List<PerContainerLogFileInfo> logMeta = responseText.get(0)
+    List<ContainerLogFileInfo> logMeta = responseText.get(0)
         .getContainerLogsInfo();
     assertTrue(logMeta.size() == 1);
     assertEquals(logMeta.get(0).getFileName(), fileName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java
index bc3ab39..1bb0408 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java
@@ -27,14 +27,14 @@ import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
-import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo;
 
 /**
  * {@code ContainerLogsInfo} includes the log meta-data of containers.
  * <p>
  * The container log meta-data includes details such as:
  * <ul>
- *   <li>A list of {@link PerContainerLogFileInfo}.</li>
+ *   <li>A list of {@link ContainerLogFileInfo}.</li>
  *   <li>The container Id.</li>
  *   <li>The NodeManager Id.</li>
  *   <li>The logType: could be local or aggregated</li>
@@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
 public class ContainerLogsInfo {
 
   @XmlElement(name = "containerLogInfo")
-  protected List<PerContainerLogFileInfo> containerLogsInfo;
+  protected List<ContainerLogFileInfo> containerLogsInfo;
 
   @XmlElement(name = "logAggregationType")
   protected String logType;
@@ -62,14 +62,14 @@ public class ContainerLogsInfo {
 
   public ContainerLogsInfo(ContainerLogMeta logMeta,
       ContainerLogAggregationType logType) throws YarnException {
-    this.containerLogsInfo = new ArrayList<PerContainerLogFileInfo>(
+    this.containerLogsInfo = new ArrayList<ContainerLogFileInfo>(
         logMeta.getContainerLogMeta());
     this.logType = logType.toString();
     this.containerId = logMeta.getContainerId();
     this.nodeId = logMeta.getNodeId();
   }
 
-  public List<PerContainerLogFileInfo> getContainerLogsInfo() {
+  public List<ContainerLogFileInfo> getContainerLogsInfo() {
     return this.containerLogsInfo;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java
index 5415e04..193ec62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java
@@ -27,7 +27,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
-import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.ContainerLogsUtils;
 import org.apache.hadoop.yarn.server.webapp.dao.ContainerLogsInfo;
@@ -55,10 +55,10 @@ public class NMContainerLogsInfo extends ContainerLogsInfo {
         containerId, remoteUser, nmContext);
   }
 
-  private static List<PerContainerLogFileInfo> getContainerLogsInfo(
+  private static List<ContainerLogFileInfo> getContainerLogsInfo(
       ContainerId id, String remoteUser, Context nmContext)
       throws YarnException {
-    List<PerContainerLogFileInfo> logFiles = new ArrayList<>();
+    List<ContainerLogFileInfo> logFiles = new ArrayList<>();
     List<File> logDirs = ContainerLogsUtils.getContainerLogDirs(
         id, remoteUser, nmContext);
     for (File containerLogsDir : logDirs) {
@@ -66,7 +66,7 @@ public class NMContainerLogsInfo extends ContainerLogsInfo {
       if (logs != null) {
         for (File log : logs) {
           if (log.isFile()) {
-            PerContainerLogFileInfo logMeta = new PerContainerLogFileInfo(
+            ContainerLogFileInfo logMeta = new ContainerLogFileInfo(
                 log.getName(), Long.toString(log.length()),
                 Times.format(log.lastModified()));
             logFiles.add(logMeta);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fddabc2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
index 1641171..fbab34a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
-import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo;
 import org.apache.hadoop.yarn.logaggregation.TestContainerLogsUtils;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
@@ -537,7 +537,7 @@ public class TestNMWebServices extends JerseyTestBase {
     assertTrue(responseList.size() == 1);
     assertEquals(responseList.get(0).getLogType(),
         ContainerLogAggregationType.LOCAL.toString());
-    List<PerContainerLogFileInfo> logMeta = responseList.get(0)
+    List<ContainerLogFileInfo> logMeta = responseList.get(0)
         .getContainerLogsInfo();
     assertTrue(logMeta.size() == 1);
     assertEquals(logMeta.get(0).getFileName(), filename);
@@ -564,13 +564,13 @@ public class TestNMWebServices extends JerseyTestBase {
       for (ContainerLogsInfo logInfo : responseList) {
         if(logInfo.getLogType().equals(
             ContainerLogAggregationType.AGGREGATED.toString())) {
-          List<PerContainerLogFileInfo> meta = logInfo.getContainerLogsInfo();
+          List<ContainerLogFileInfo> meta = logInfo.getContainerLogsInfo();
           assertTrue(meta.size() == 1);
           assertEquals(meta.get(0).getFileName(), aggregatedLogFile);
         } else {
           assertEquals(logInfo.getLogType(),
               ContainerLogAggregationType.LOCAL.toString());
-          List<PerContainerLogFileInfo> meta = logInfo.getContainerLogsInfo();
+          List<ContainerLogFileInfo> meta = logInfo.getContainerLogsInfo();
           assertTrue(meta.size() == 1);
           assertEquals(meta.get(0).getFileName(), filename);
         }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org