Browse Source

YARN-10215. Endpoint for obtaining direct URL for the logs. Contributed by Andras Gyori

Szilard Nemeth 5 years ago
parent
commit
ab36429559
30 changed files with 702 additions and 55 deletions
  1. 12 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
  2. 14 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
  3. 49 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogServlet.java
  4. 12 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogWebService.java
  5. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/YarnWebServiceParams.java
  6. 9 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-jhs-log.js
  7. 59 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-jhs-redirect-log.js
  8. 9 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-log.js
  9. 60 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-redirect-log.js
  10. 11 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-log.js
  11. 30 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-redirect-log.js
  12. 23 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-log.js
  13. 30 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-redirect-log.js
  14. 35 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/logs.js
  15. 30 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/log-adapter-helper.js
  16. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app-jhs-log.js
  17. 26 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app-jhs-redirect-log.js
  18. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app-log.js
  19. 26 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app-redirect-log.js
  20. 6 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-jhs-log.js
  21. 28 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-jhs-redirect-log.js
  22. 6 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-log.js
  23. 28 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-redirect-log.js
  24. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app-jhs-log.js
  25. 36 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app-jhs-redirect-log.js
  26. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app-log.js
  27. 36 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app-redirect-log.js
  28. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-jhs-log.js
  29. 57 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-jhs-redirect-log.js
  30. 54 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-redirect-log.js

+ 12 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java

@@ -434,10 +434,12 @@ public class HsWebServices extends WebServices {
       @QueryParam(YarnWebServiceParams.CONTAINER_ID) String containerIdStr,
       @QueryParam(YarnWebServiceParams.NM_ID) String nmId,
       @QueryParam(YarnWebServiceParams.REDIRECTED_FROM_NODE)
-      @DefaultValue("false") boolean redirectedFromNode) {
+      @DefaultValue("false") boolean redirectedFromNode,
+      @QueryParam(YarnWebServiceParams.MANUAL_REDIRECTION)
+      @DefaultValue("false") boolean manualRedirection) {
     init();
     return logServlet.getLogsInfo(hsr, appIdStr, appAttemptIdStr,
-        containerIdStr, nmId, redirectedFromNode);
+        containerIdStr, nmId, redirectedFromNode, manualRedirection);
   }
 
   @GET
@@ -449,14 +451,16 @@ public class HsWebServices extends WebServices {
       @PathParam(YarnWebServiceParams.CONTAINER_ID) String containerIdStr,
       @QueryParam(YarnWebServiceParams.NM_ID) String nmId,
       @QueryParam(YarnWebServiceParams.REDIRECTED_FROM_NODE)
-      @DefaultValue("false") boolean redirectedFromNode) {
+      @DefaultValue("false") boolean redirectedFromNode,
+      @QueryParam(YarnWebServiceParams.MANUAL_REDIRECTION)
+      @DefaultValue("false") boolean manualRedirection) {
     init();
 
     WrappedLogMetaRequest.Builder logMetaRequestBuilder =
         LogServlet.createRequestFromContainerId(containerIdStr);
 
     return logServlet.getContainerLogsInfo(hsr, logMetaRequestBuilder, nmId,
-        redirectedFromNode, null);
+        redirectedFromNode, null, manualRedirection);
   }
 
   @GET
@@ -474,10 +478,12 @@ public class HsWebServices extends WebServices {
           String size,
       @QueryParam(YarnWebServiceParams.NM_ID) String nmId,
       @QueryParam(YarnWebServiceParams.REDIRECTED_FROM_NODE)
-      @DefaultValue("false") boolean redirectedFromNode) {
+      @DefaultValue("false") boolean redirectedFromNode,
+      @QueryParam(YarnWebServiceParams.MANUAL_REDIRECTION)
+      @DefaultValue("false") boolean manualRedirection) {
     init();
     return logServlet.getLogFile(req, containerIdStr, filename, format, size,
-        nmId, redirectedFromNode, null);
+        nmId, redirectedFromNode, null, manualRedirection);
   }
 
   @VisibleForTesting

+ 14 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java

@@ -210,7 +210,7 @@ public class AHSWebServices extends WebServices {
    *    The container ID
    * @param nmId
    *    The Node Manager NodeId
-   * @param redirected_from_node
+   * @param redirectedFromNode
    *    Whether this is a redirected request from NM
    * @return
    *    The log file's name and current file size
@@ -224,14 +224,16 @@ public class AHSWebServices extends WebServices {
       @PathParam(YarnWebServiceParams.CONTAINER_ID) String containerIdStr,
       @QueryParam(YarnWebServiceParams.NM_ID) String nmId,
       @QueryParam(YarnWebServiceParams.REDIRECTED_FROM_NODE)
-      @DefaultValue("false") boolean redirected_from_node) {
+      @DefaultValue("false") boolean redirectedFromNode,
+      @QueryParam(YarnWebServiceParams.MANUAL_REDIRECTION)
+      @DefaultValue("false") boolean manualRedirection) {
     initForReadableEndpoints(res);
 
     WrappedLogMetaRequest.Builder logMetaRequestBuilder =
         LogServlet.createRequestFromContainerId(containerIdStr);
 
     return logServlet.getContainerLogsInfo(req, logMetaRequestBuilder, nmId,
-        redirected_from_node, null);
+        redirectedFromNode, null, manualRedirection);
   }
 
   /**
@@ -251,7 +253,7 @@ public class AHSWebServices extends WebServices {
    *    the size of the log file
    * @param nmId
    *    The Node Manager NodeId
-   * @param redirected_from_node
+   * @param redirectedFromNode
    *    Whether this is the redirect request from NM
    * @return
    *    The contents of the container's log file
@@ -269,9 +271,11 @@ public class AHSWebServices extends WebServices {
       @QueryParam(YarnWebServiceParams.RESPONSE_CONTENT_SIZE) String size,
       @QueryParam(YarnWebServiceParams.NM_ID) String nmId,
       @QueryParam(YarnWebServiceParams.REDIRECTED_FROM_NODE)
-      boolean redirected_from_node) {
+      boolean redirectedFromNode,
+      @QueryParam(YarnWebServiceParams.MANUAL_REDIRECTION)
+      @DefaultValue("false") boolean manualRedirection) {
     return getLogs(req, res, containerIdStr, filename, format,
-        size, nmId, redirected_from_node);
+        size, nmId, redirectedFromNode, manualRedirection);
   }
 
   //TODO: YARN-4993: Refactory ContainersLogsBlock, AggregatedLogsBlock and
@@ -290,10 +294,12 @@ public class AHSWebServices extends WebServices {
       @QueryParam(YarnWebServiceParams.RESPONSE_CONTENT_SIZE) String size,
       @QueryParam(YarnWebServiceParams.NM_ID) String nmId,
       @QueryParam(YarnWebServiceParams.REDIRECTED_FROM_NODE)
-      @DefaultValue("false") boolean redirected_from_node) {
+      @DefaultValue("false") boolean redirectedFromNode,
+      @QueryParam(YarnWebServiceParams.MANUAL_REDIRECTION)
+      @DefaultValue("false") boolean manualRedirection) {
     initForReadableEndpoints(res);
     return logServlet.getLogFile(req, containerIdStr, filename, format, size,
-        nmId, redirected_from_node, null);
+        nmId, redirectedFromNode, null, manualRedirection);
   }
 
   @VisibleForTesting

+ 49 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogServlet.java

@@ -44,7 +44,10 @@ import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.GenericEntity;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status;
+import javax.ws.rs.core.StreamingOutput;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 /**
@@ -52,6 +55,7 @@ import java.util.List;
  * Used by various WebServices (AHS, ATS).
  */
 public class LogServlet extends Configured {
+
   private static final Logger LOG = LoggerFactory
       .getLogger(LogServlet.class);
 
@@ -164,7 +168,7 @@ public class LogServlet extends Configured {
 
   public Response getLogsInfo(HttpServletRequest hsr, String appIdStr,
       String appAttemptIdStr, String containerIdStr, String nmId,
-      boolean redirectedFromNode) {
+      boolean redirectedFromNode, boolean manualRedirection) {
     ApplicationId appId = null;
     if (appIdStr != null) {
       try {
@@ -201,9 +205,10 @@ public class LogServlet extends Configured {
             .setContainerId(containerIdStr);
 
     return getContainerLogsInfo(hsr, logMetaRequestBuilder, nmId,
-        redirectedFromNode, null);
+        redirectedFromNode, null, manualRedirection);
   }
 
+
   /**
    * Returns information about the logs for a specific container.
    *
@@ -212,12 +217,14 @@ public class LogServlet extends Configured {
    * @param nmId NodeManager id
    * @param redirectedFromNode whether the request was redirected
    * @param clusterId the id of the cluster
+   * @param manualRedirection whether to return a response with a Location
+   *                          instead of an automatic redirection
    * @return {@link Response} object containing information about the logs
    */
   public Response getContainerLogsInfo(HttpServletRequest req,
       WrappedLogMetaRequest.Builder builder,
       String nmId, boolean redirectedFromNode,
-      String clusterId) {
+      String clusterId, boolean manualRedirection) {
 
     builder.setFactory(factory);
 
@@ -287,6 +294,10 @@ public class LogServlet extends Configured {
       if (query != null && !query.isEmpty()) {
         resURI += "?" + query;
       }
+      if (manualRedirection) {
+        return createLocationResponse(resURI, createEmptyLogsInfo());
+      }
+
       Response.ResponseBuilder response = Response.status(
           HttpServletResponse.SC_TEMPORARY_REDIRECT);
       response.header("Location", resURI);
@@ -297,6 +308,32 @@ public class LogServlet extends Configured {
     }
   }
 
+  /**
+   * Creates a response with empty payload and a location header to preserve
+   * API compatibility.
+   *
+   * @param uri redirection url
+   * @param emptyPayload a payload that is discarded
+   * @return a response with empty payload
+   */
+  private static <T> Response createLocationResponse(
+      String uri, T emptyPayload) {
+    Response.ResponseBuilder response = Response.status(
+        HttpServletResponse.SC_OK).entity(emptyPayload);
+    response.header("Location", uri);
+    response.header("Access-Control-Expose-Headers", "Location");
+    return response.build();
+  }
+
+  private static GenericEntity<List<ContainerLogsInfo>> createEmptyLogsInfo() {
+    return new GenericEntity<List<ContainerLogsInfo>>(
+        Collections.EMPTY_LIST, List.class);
+  }
+
+  private static StreamingOutput createEmptyStream() {
+    return outputStream -> outputStream.write(
+        "".getBytes(Charset.defaultCharset()));
+  }
 
   /**
    * Returns an aggregated log file belonging to a container.
@@ -309,11 +346,13 @@ public class LogServlet extends Configured {
    * @param nmId NodeManager id
    * @param redirectedFromNode whether the request was redirected
    * @param clusterId the id of the cluster
+   * @param manualRedirection whether to return a response with a Location
+   *                          instead of an automatic redirection
    * @return {@link Response} object containing information about the logs
    */
   public Response getLogFile(HttpServletRequest req, String containerIdStr,
       String filename, String format, String size, String nmId,
-      boolean redirectedFromNode, String clusterId) {
+      boolean redirectedFromNode, String clusterId, boolean manualRedirection) {
     ContainerId containerId;
     try {
       containerId = ContainerId.fromString(containerIdStr);
@@ -388,6 +427,12 @@ public class LogServlet extends Configured {
       if (query != null && !query.isEmpty()) {
         resURI += "?" + query;
       }
+
+
+      if (manualRedirection) {
+        return createLocationResponse(resURI, createEmptyStream());
+      }
+
       Response.ResponseBuilder response = Response.status(
           HttpServletResponse.SC_TEMPORARY_REDIRECT);
       response.header("Location", resURI);

+ 12 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/LogWebService.java

@@ -149,14 +149,16 @@ public class LogWebService implements AppInfoProvider {
       @QueryParam(YarnWebServiceParams.NM_ID) String nmId,
       @QueryParam(YarnWebServiceParams.REDIRECTED_FROM_NODE)
       @DefaultValue("false") boolean redirectedFromNode,
-      @QueryParam(YarnWebServiceParams.CLUSTER_ID) String clusterId) {
+      @QueryParam(YarnWebServiceParams.CLUSTER_ID) String clusterId,
+      @QueryParam(YarnWebServiceParams.MANUAL_REDIRECTION)
+      @DefaultValue("false") boolean manualRedirection) {
     initForReadableEndpoints(res);
 
     WrappedLogMetaRequest.Builder logMetaRequestBuilder =
         LogServlet.createRequestFromContainerId(containerIdStr);
 
     return logServlet.getContainerLogsInfo(req, logMetaRequestBuilder, nmId,
-        redirectedFromNode, clusterId);
+        redirectedFromNode, clusterId, manualRedirection);
   }
 
   @Override
@@ -256,9 +258,11 @@ public class LogWebService implements AppInfoProvider {
       @QueryParam(YarnWebServiceParams.NM_ID) String nmId,
       @QueryParam(YarnWebServiceParams.REDIRECTED_FROM_NODE)
           boolean redirectedFromNode,
-      @QueryParam(YarnWebServiceParams.CLUSTER_ID) String clusterId) {
+      @QueryParam(YarnWebServiceParams.CLUSTER_ID) String clusterId,
+      @QueryParam(YarnWebServiceParams.MANUAL_REDIRECTION)
+      @DefaultValue("false") boolean manualRedirection) {
     return getLogs(req, res, containerIdStr, filename, format, size, nmId,
-        redirectedFromNode, clusterId);
+        redirectedFromNode, clusterId, manualRedirection);
   }
 
   //TODO: YARN-4993: Refactory ContainersLogsBlock, AggregatedLogsBlock and
@@ -278,10 +282,12 @@ public class LogWebService implements AppInfoProvider {
       @QueryParam(YarnWebServiceParams.NM_ID) String nmId,
       @QueryParam(YarnWebServiceParams.REDIRECTED_FROM_NODE)
       @DefaultValue("false") boolean redirectedFromNode,
-      @QueryParam(YarnWebServiceParams.CLUSTER_ID) String clusterId) {
+      @QueryParam(YarnWebServiceParams.CLUSTER_ID) String clusterId,
+      @QueryParam(YarnWebServiceParams.MANUAL_REDIRECTION)
+      @DefaultValue("false") boolean manualRedirection) {
     initForReadableEndpoints(res);
     return logServlet.getLogFile(req, containerIdStr, filename, format, size,
-        nmId, redirectedFromNode, clusterId);
+        nmId, redirectedFromNode, clusterId, manualRedirection);
   }
 
   @VisibleForTesting protected TimelineEntity getEntity(String path,

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/YarnWebServiceParams.java

@@ -38,4 +38,5 @@ public interface YarnWebServiceParams {
   String NM_ID = "nm.id";
   String REDIRECTED_FROM_NODE = "redirected_from_node";
   String CLUSTER_ID = "clusterid";
+  String MANUAL_REDIRECTION = "manual_redirection";
 }

+ 9 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-jhs-log.js

@@ -38,11 +38,19 @@ export default RESTAbstractAdapter.extend({
     var containerId = splits[0];
     var logFile = splits[1];
     var url = this._buildURL();
-    url = url + '/containerlogs/' + containerId + '/' + logFile;
+    url = url + '/containerlogs/' + containerId + '/' + logFile + '?manual_redirection=true';
     Ember.Logger.info('The URL for getting the log: ' + url);
     return url;
   },
 
+  handleResponse(status, headers, payload, requestData) {
+    if (headers['location'] !== undefined && headers['location'] !== null) {
+      return { redirectedUrl: headers.location, data: "" }
+    } else {
+      return { data: payload }
+    }
+  },
+
   /**
    * Override options so that result is not expected to be JSON
    */

+ 59 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-jhs-redirect-log.js

@@ -0,0 +1,59 @@
+/**
+ * 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.
+ */
+
+import Ember from 'ember';
+import Converter from 'yarn-ui/utils/converter';
+import RESTAbstractAdapter from './restabstract';
+
+/**
+ * REST URL's response when fetching container logs will be
+ * in plain text format and not JSON.
+ */
+export default RESTAbstractAdapter.extend({
+  headers: {
+    Accept: 'text/plain'
+  },
+
+  urlForFindRecord(id/*, modelName, snapshot*/) {
+    var splits = Converter.splitForAppLogs(id);
+    var url = splits[0];
+    return url;
+  },
+
+  /**
+   * Override options so that result is not expected to be JSON
+   */
+  ajaxOptions: function (url, type, options) {
+    var hash = options || {};
+    hash.url = url;
+    hash.type = type;
+    // Make sure jQuery does not try to convert response to JSON.
+    hash.dataType = 'text';
+    hash.context = this;
+
+    var headers = Ember.get(this, 'headers');
+    if (headers !== undefined) {
+      hash.beforeSend = function (xhr) {
+        Object.keys(headers).forEach(function (key) {
+          return xhr.setRequestHeader(key, headers[key]);
+        });
+      };
+    }
+    return hash;
+  }
+});

+ 9 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-log.js

@@ -42,11 +42,19 @@ export default RESTAbstractAdapter.extend({
       clusterId = splits[2];
     }
     var url = this._buildURL();
-    url = url + '/containers/' + containerId + '/logs/' + logFile + '?clusterid=' + clusterId;
+    url = url + '/containers/' + containerId + '/logs/' + logFile + '?clusterid=' + clusterId + '?manual_redirection=true';
     Ember.Logger.info('The URL for getting the log: ' + url);
     return url;
   },
 
+  handleResponse(status, headers, payload, requestData) {
+    if (headers['location'] !== undefined && headers['location'] !== null) {
+      return { redirectedUrl: headers.location, data: "" }
+    } else {
+      return { data: payload }
+    }
+  },
+
   /**
    * Override options so that result is not expected to be JSON
    */

+ 60 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-redirect-log.js

@@ -0,0 +1,60 @@
+/**
+ * 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.
+ */
+
+import Ember from 'ember';
+import Converter from 'yarn-ui/utils/converter';
+import RESTAbstractAdapter from './restabstract';
+
+/**
+ * REST URL's response when fetching container logs will be
+ * in plain text format and not JSON.
+ */
+export default RESTAbstractAdapter.extend({
+
+  headers: {
+    Accept: 'text/plain'
+  },
+
+  urlForFindRecord(id/*, modelName, snapshot*/) {
+    var splits = Converter.splitForAppLogs(id);
+    var url = splits[0];
+    return url;
+  },
+
+  /**
+   * Override options so that result is not expected to be JSON
+   */
+  ajaxOptions: function (url, type, options) {
+    var hash = options || {};
+    hash.url = url;
+    hash.type = type;
+    // Make sure jQuery does not try to convert response to JSON.
+    hash.dataType = 'text';
+    hash.context = this;
+
+    var headers = Ember.get(this, 'headers');
+    if (headers !== undefined) {
+      hash.beforeSend = function (xhr) {
+        Object.keys(headers).forEach(function (key) {
+          return xhr.setRequestHeader(key, headers[key]);
+        });
+      };
+    }
+    return hash;
+  }
+});

+ 11 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-log.js

@@ -17,6 +17,7 @@
  */
 
 import AbstractAdapter from './abstract';
+import { createEmptyContainerLogInfo } from 'yarn-ui/helpers/log-adapter-helper';
 
 export default AbstractAdapter.extend({
   address: "jhsAddress",
@@ -27,6 +28,15 @@ export default AbstractAdapter.extend({
     var url = this._buildURL();
     var containerId = query['containerId'];
     delete query.containerId;
-    return url + '/containers/' + containerId + '/logs';
+    return url + '/containers/' + containerId + '/logs' + '?manual_redirection=true';
+  },
+
+  handleResponse(status, headers, payload, requestData) {
+    if (headers['location'] !== undefined && headers['location'] !== null) {
+      return createEmptyContainerLogInfo(headers['location']);
+    } else {
+      return payload;
+    }
   }
+
 });

+ 30 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-redirect-log.js

@@ -0,0 +1,30 @@
+/**
+ * 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.
+ */
+
+import AbstractAdapter from './abstract';
+
+export default AbstractAdapter.extend({
+  address: "jhsAddress",
+  restNameSpace: "jhs",
+  serverName: "JHS",
+
+  urlForQuery(url/*, modelName*/) {
+    return url;
+  }
+
+});

+ 23 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-log.js

@@ -18,6 +18,19 @@
 
 import AbstractAdapter from './abstract';
 
+function createEmptyContainerLogInfo(location) {
+  return {
+    containerLogsInfo: {
+      containerLogInfo: [{
+        fileName: "",
+        fileSize: "",
+        lastModifiedTime: "",
+        redirectedUrl: location
+      }]
+    }
+  };
+}
+
 export default AbstractAdapter.extend({
   address: "timelineWebAddress",
   restNameSpace: "timelineV2Log",
@@ -28,6 +41,15 @@ export default AbstractAdapter.extend({
     var containerId = query['containerId'];
     var clusterId = this.get("env.app.clusterId");
     delete query.containerId;
-    return url + '/containers/' + containerId + '/logs' + '?clusterid=' + clusterId;
+    return url + '/containers/' + containerId + '/logs' + '?clusterid=' + clusterId + '?manual_redirection=true';
+  },
+
+  handleResponse(status, headers, payload, requestData) {
+    if (headers['location'] !== undefined && headers['location'] !== null) {
+      return createEmptyContainerLogInfo(headers['location']);
+    } else {
+      return payload;
+    }
   }
+
 });

+ 30 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-redirect-log.js

@@ -0,0 +1,30 @@
+/**
+ * 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.
+ */
+
+import AbstractAdapter from './abstract';
+
+export default AbstractAdapter.extend({
+  address: "timelineWebAddress",
+  restNameSpace: "timelineV2Log",
+  serverName: "ATS",
+
+  urlForQuery(url/*, modelName*/) {
+    return url;
+  }
+
+});

+ 35 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/logs.js

@@ -235,26 +235,54 @@ export default Ember.Controller.extend({
 
   fetchLogFilesForContainerId(containerId) {
     let queryName = this.fallbackToJHS ? "yarn-jhs-log" : "yarn-log";
+    let redirectQuery = queryName === "yarn-jhs-log" ? "yarn-jhs-redirect-log" : "yarn-redirect-log";
 
     return Ember.RSVP.hash({
-      logs: this.store
-        .query(queryName, {
-          containerId: containerId
-        })
-        .catch(function() {
-          return Ember.A();
+      logs: this.resolveRedirectableQuery(
+        this.store.query(queryName, { containerId }),
+        m => {
+          return m.map(model => model.get('redirectedUrl'))[0];
+        },
+        url => {
+          return this.store.query(redirectQuery, url);
         })
     });
   },
 
   fetchContentForLogFile(id) {
     let queryName = this.fallbackToJHS ? 'yarn-app-jhs-log' : 'yarn-app-log';
+    let redirectQuery = queryName === "yarn-app-jhs-log" ? "yarn-app-jhs-redirect-log" : "yarn-app-redirect-log";
 
     return Ember.RSVP.hash({
-      logs: this.store.findRecord(queryName, id)
+      logs: this.resolveRedirectableQuery(
+        this.store.findRecord(queryName, id),
+        m => {
+          return m.get('redirectedUrl');
+        },
+        url => {
+          return this.store.findRecord(redirectQuery, url + Constants.PARAM_SEPARATOR + id);
+        })
     });
   },
 
+  resolveRedirectableQuery(initial, urlResolver, redirectResolver) {
+    return initial.then(m => {
+      let redirectedUrl = urlResolver(m);
+      if (redirectedUrl !== null && redirectedUrl !== undefined && redirectedUrl !== '') {
+        let logFromRedirect = redirectResolver(redirectedUrl);
+        return Promise.all([m, logFromRedirect]);
+      } else {
+        return Promise.all([m, null]);
+      }
+    })
+      .then(([originalLog, logFromRedirect]) => {
+        return logFromRedirect !== null ? logFromRedirect : originalLog;
+      })
+      .catch(function () {
+        return Ember.A();
+      });
+  },
+
   resetAfterRefresh() {
     this.set("selectedAttemptId", "");
     this.set("attemptContainerList", null);

+ 30 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/log-adapter-helper.js

@@ -0,0 +1,30 @@
+/**
+ * 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.
+ */
+
+export function createEmptyContainerLogInfo(location) {
+  return {
+    containerLogsInfo: {
+      containerLogInfo: [{
+        fileName: "",
+        fileSize: "",
+        lastModifiedTime: "",
+        redirectedUrl: location
+      }]
+    }
+  };
+}

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app-jhs-log.js

@@ -19,7 +19,8 @@
 import DS from 'ember-data';
 
 export default DS.Model.extend({
-  logs: DS.attr('string'),
-  containerID: DS.attr('string'),
-  logFileName: DS.attr('string')
+  logs: DS.attr('string', {defaultValue: ''}),
+  containerID: DS.attr('string', {defaultValue: ''}),
+  logFileName: DS.attr('string', {defaultValue: ''}),
+  redirectedUrl: DS.attr('string', {defaultValue: ''}),
 });

+ 26 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app-jhs-redirect-log.js

@@ -0,0 +1,26 @@
+/**
+ * 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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  logs: DS.attr('string', {defaultValue: ''}),
+  containerID: DS.attr('string', {defaultValue: ''}),
+  logFileName: DS.attr('string', {defaultValue: ''}),
+  redirectedUrl: DS.attr('string', {defaultValue: ''}),
+});

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app-log.js

@@ -19,7 +19,8 @@
 import DS from 'ember-data';
 
 export default DS.Model.extend({
-  logs: DS.attr('string'),
-  containerID: DS.attr('string'),
-  logFileName: DS.attr('string')
+  logs: DS.attr('string', {defaultValue: ''}),
+  containerID: DS.attr('string', {defaultValue: ''}),
+  logFileName: DS.attr('string', {defaultValue: ''}),
+  redirectedUrl: DS.attr('string', {defaultValue: ''}),
 });

+ 26 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app-redirect-log.js

@@ -0,0 +1,26 @@
+/**
+ * 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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  logs: DS.attr('string', {defaultValue: ''}),
+  containerID: DS.attr('string', {defaultValue: ''}),
+  logFileName: DS.attr('string', {defaultValue: ''}),
+  redirectedUrl: DS.attr('string', {defaultValue: ''}),
+});

+ 6 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-jhs-log.js

@@ -19,9 +19,10 @@
 import DS from 'ember-data';
 
 export default DS.Model.extend({
-  fileName: DS.attr('string'),
-  fileSize: DS.attr('string'),
-  lastModifiedTime: DS.attr('string'),
-  containerId: DS.attr('string'),
-  nodeId: DS.attr('string')
+  fileName: DS.attr('string', {defaultValue: ''}),
+  fileSize: DS.attr('string', {defaultValue: ''}),
+  lastModifiedTime: DS.attr('string', {defaultValue: ''}),
+  containerId: DS.attr('string', {defaultValue: ''}),
+  nodeId: DS.attr('string', {defaultValue: ''}),
+  redirectedUrl: DS.attr('string', {defaultValue: ''})
 });

+ 28 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-jhs-redirect-log.js

@@ -0,0 +1,28 @@
+/**
+ * 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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  fileName: DS.attr('string', {defaultValue: ''}),
+  fileSize: DS.attr('string', {defaultValue: ''}),
+  lastModifiedTime: DS.attr('string', {defaultValue: ''}),
+  containerId: DS.attr('string', {defaultValue: ''}),
+  nodeId: DS.attr('string', {defaultValue: ''}),
+  redirectedUrl: DS.attr('string', {defaultValue: ''})
+});

+ 6 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-log.js

@@ -19,9 +19,10 @@
 import DS from 'ember-data';
 
 export default DS.Model.extend({
-  fileName: DS.attr('string'),
-  fileSize: DS.attr('string'),
-  lastModifiedTime: DS.attr('string'),
-  containerId: DS.attr('string'),
-  nodeId: DS.attr('string')
+  fileName: DS.attr('string', {defaultValue: ''}),
+  fileSize: DS.attr('string', {defaultValue: ''}),
+  lastModifiedTime: DS.attr('string', {defaultValue: ''}),
+  containerId: DS.attr('string', {defaultValue: ''}),
+  nodeId: DS.attr('string', {defaultValue: ''}),
+  redirectedUrl: DS.attr('string', {defaultValue: ''})
 });

+ 28 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-redirect-log.js

@@ -0,0 +1,28 @@
+/**
+ * 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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  fileName: DS.attr('string', {defaultValue: ''}),
+  fileSize: DS.attr('string', {defaultValue: ''}),
+  lastModifiedTime: DS.attr('string', {defaultValue: ''}),
+  containerId: DS.attr('string', {defaultValue: ''}),
+  nodeId: DS.attr('string', {defaultValue: ''}),
+  redirectedUrl: DS.attr('string', {defaultValue: ''})
+});

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app-jhs-log.js

@@ -28,9 +28,10 @@ export default DS.JSONAPISerializer.extend({
       id: id,
       type: primaryModelClass.modelName,
       attributes: {
-        logs: payload,
+        logs: payload.data,
         containerID: splits[0],
-        logFileName: splits[1]
+        logFileName: splits[1],
+        redirectedUrl: payload.redirectedUrl
       }
     };
     return { data: convertedPayload };

+ 36 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app-jhs-redirect-log.js

@@ -0,0 +1,36 @@
+/**
+ * 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.
+ */
+
+import DS from 'ember-data';
+import Converter from 'yarn-ui/utils/converter';
+
+export default DS.JSONAPISerializer.extend({
+  normalizeSingleResponse(store, primaryModelClass, payload, id/*, requestType*/) {
+    var splits = Converter.splitForAppLogs(id);
+    var convertedPayload = {
+      id: id,
+      type: primaryModelClass.modelName,
+      attributes: {
+        logs: payload,
+        containerID: splits[1],
+        logFileName: splits[2]
+      }
+    };
+    return { data: convertedPayload };
+  },
+});

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app-log.js

@@ -28,9 +28,10 @@ export default DS.JSONAPISerializer.extend({
       id: id,
       type: primaryModelClass.modelName,
       attributes: {
-        logs: payload,
+        logs: payload.data,
         containerID: splits[0],
-        logFileName: splits[1]
+        logFileName: splits[1],
+        redirectedUrl: payload.redirectedUrl
       }
     };
     return { data: convertedPayload };

+ 36 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app-redirect-log.js

@@ -0,0 +1,36 @@
+/**
+ * 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.
+ */
+
+import DS from 'ember-data';
+import Converter from 'yarn-ui/utils/converter';
+
+export default DS.JSONAPISerializer.extend({
+  normalizeSingleResponse(store, primaryModelClass, payload, id/*, requestType*/) {
+    var splits = Converter.splitForAppLogs(id);
+    var convertedPayload = {
+      id: id,
+      type: primaryModelClass.modelName,
+      attributes: {
+        logs: payload,
+        containerID: splits[1],
+        logFileName: splits[2]
+      }
+    };
+    return { data: convertedPayload };
+  },
+});

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-jhs-log.js

@@ -26,6 +26,7 @@ export default DS.JSONAPISerializer.extend({
         fileName: payload.fileName,
         fileSize: payload.fileSize,
         lastModifiedTime: payload.lastModifiedTime,
+        redirectedUrl: payload.redirectedUrl,
         containerId: containerId,
         nodeId: nodeId
       }

+ 57 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-jhs-redirect-log.js

@@ -0,0 +1,57 @@
+/**
+ * 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.
+ */
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+  internalNormalizeSingleResponse(store, primaryModelClass, payload, containerId, nodeId) {
+    var fixedPayload = {
+      id: "yarn_log_" + payload.fileName + "_" + Date.now(),
+      type: primaryModelClass.modelName,
+      attributes: {
+        fileName: payload.fileName,
+        fileSize: payload.fileSize,
+        lastModifiedTime: payload.lastModifiedTime,
+        redirectedUrl: payload.redirectedUrl,
+        containerId: containerId,
+        nodeId: nodeId
+      }
+    };
+    return fixedPayload;
+  },
+
+  normalizeArrayResponse(store, primaryModelClass, payload/*, id, requestType*/) {
+    var normalizedArrayResponse = {
+      data: []
+    };
+    // If JSON payload is an object with a containerLogsInfo property
+    if (payload && payload.containerLogsInfo && payload.containerLogsInfo.containerLogInfo) {
+      normalizedArrayResponse.data = payload.containerLogsInfo.containerLogInfo.map((signle_payload) => {
+        return this.internalNormalizeSingleResponse(store, primaryModelClass, signle_payload,
+          payload.containerLogsInfo.containerId, payload.containerLogsInfo.nodeId);
+      });
+    }
+    // If JSON payload is an array
+    if (payload && payload[0] && payload[0].containerLogInfo) {
+      normalizedArrayResponse.data = payload[0].containerLogInfo.map((signle_payload) => {
+        return this.internalNormalizeSingleResponse(store, primaryModelClass, signle_payload,
+          payload[0].containerId, payload[0].nodeId);
+      });
+    }
+    return normalizedArrayResponse;
+  }
+});

+ 54 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-redirect-log.js

@@ -0,0 +1,54 @@
+/**
+ * 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.
+ */
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+  internalNormalizeSingleResponse(store, primaryModelClass, payload, containerId, nodeId) {
+    var fixedPayload = {
+      id: "yarn_log_" + payload.fileName + "_" + Date.now(),
+      type: primaryModelClass.modelName,
+      attributes: {
+        fileName: payload.fileName,
+        fileSize: payload.fileSize,
+        lastModifiedTime: payload.lastModifiedTime,
+        containerId: containerId,
+        nodeId: nodeId
+      }
+    };
+    return fixedPayload;
+  },
+
+  normalizeArrayResponse(store, primaryModelClass, payload/*, id, requestType*/) {
+    var normalizedArrayResponse = {
+      data: []
+    };
+    if (payload && payload.containerLogsInfo && payload.containerLogsInfo.containerLogInfo) {
+      normalizedArrayResponse.data = payload.containerLogsInfo.containerLogInfo.map((paylog) => {
+        return this.internalNormalizeSingleResponse(store, primaryModelClass, paylog,
+          payload.containerLogsInfo.containerId, payload.containerLogsInfo.nodeId);
+      });
+    }
+    if (payload && payload[0] && payload[0].containerLogInfo) {
+      normalizedArrayResponse.data = payload[0].containerLogInfo.map((paylog) => {
+        return this.internalNormalizeSingleResponse(store, primaryModelClass, paylog,
+          payload[0].containerId, payload[0].nodeId);
+      });
+    }
+    return normalizedArrayResponse;
+  }
+});