Sfoglia il codice sorgente

YARN-10029. Add option to UIv2 to get container logs from the new JHS API. Contributed by Adam Antal

Szilard Nemeth 5 anni fa
parent
commit
f1b1b332f5
21 ha cambiato i file con 607 aggiunte e 77 eliminazioni
  1. 30 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/jhs-health.js
  2. 67 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-jhs-log.js
  3. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-log.js
  4. 33 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-container.js
  5. 32 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-log.js
  6. 64 32
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/logs.js
  7. 59 40
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js
  8. 28 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/jhs-health.js
  9. 25 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app-jhs-log.js
  10. 27 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-jhs-container.js
  11. 27 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-jhs-log.js
  12. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/application.js
  13. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app/logs.js
  14. 32 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/jhs-health.js
  15. 38 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app-jhs-log.js
  16. 62 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-jhs-container.js
  17. 56 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-jhs-log.js
  18. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/services/hosts.js
  19. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app/logs.hbs
  20. 7 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/config/configs.env
  21. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/config/default-config.js

+ 30 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/jhs-health.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 RESTAbstractAdapter from './restabstract';
+
+export default RESTAbstractAdapter.extend({
+  address: "jhsAddress",
+  restNameSpace: "jhs",
+  serverName: "JHS",
+
+  urlForQueryRecord(/*query, modelName*/) {
+    var url = this.buildURL();
+    return url + '/info';
+  }
+});

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

@@ -0,0 +1,67 @@
+/**
+ * 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({
+  address: "jhsAddress",
+  restNameSpace: "jhs",
+  serverName: "JHS",
+
+  headers: {
+    Accept: 'text/plain'
+  },
+
+  urlForFindRecord(id/*, modelName, snapshot*/) {
+    var splits = Converter.splitForAppLogs(id);
+    var containerId = splits[0];
+    var logFile = splits[1];
+    var url = this._buildURL();
+    url = url + '/containerlogs/' + containerId + '/' + logFile;
+    Ember.Logger.info('The URL for getting the log: ' + url);
+    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;
+  }
+});

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

@@ -43,7 +43,7 @@ export default RESTAbstractAdapter.extend({
     }
     }
     var url = this._buildURL();
     var url = this._buildURL();
     url = url + '/containers/' + containerId + '/logs/' + logFile + '?clusterid=' + clusterId;
     url = url + '/containers/' + containerId + '/logs/' + logFile + '?clusterid=' + clusterId;
-    console.log('log url' + url);
+    Ember.Logger.info('The URL for getting the log: ' + url);
     return url;
     return url;
   },
   },
 
 

+ 33 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-container.js

@@ -0,0 +1,33 @@
+/**
+ * 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 RESTAbstractAdapter from './restabstract';
+
+export default RESTAbstractAdapter.extend({
+  address: "jhsAddress",
+  restNameSpace: "jhs",
+  serverName: "JHS",
+
+  urlForQuery(query/*, modelName*/) {
+    var url = this.buildURL();
+    url = url + '/aggregatedlogs?appattemptid=' + query.app_attempt_id;
+    delete query.app_attempt_id;
+    return url;
+  }
+});

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

@@ -0,0 +1,32 @@
+/**
+ * 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(query/*, modelName*/) {
+    var url = this._buildURL();
+    var containerId = query['containerId'];
+    delete query.containerId;
+    return url + '/containers/' + containerId + '/logs';
+  }
+});

+ 64 - 32
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/logs.js

@@ -50,31 +50,39 @@ export default Ember.Controller.extend({
         this.fetchContainersForAttemptId(attemptId)
         this.fetchContainersForAttemptId(attemptId)
           .then(hash => {
           .then(hash => {
             let containers = null;
             let containers = null;
-            let containerIdArr = [];
+            let containerIdArr = {};
+
+            // Getting running containers from the RM first
             if (
             if (
               hash.rmContainers.get("length") > 0 &&
               hash.rmContainers.get("length") > 0 &&
               hash.rmContainers.get("content")
               hash.rmContainers.get("content")
             ) {
             ) {
               hash.rmContainers.get("content").forEach(function(o) {
               hash.rmContainers.get("content").forEach(function(o) {
-                containerIdArr.push(o.id);
+                containerIdArr[o.id] = true;
               }.bind(this));
               }.bind(this));
               containers = (containers || []).concat(
               containers = (containers || []).concat(
                 hash.rmContainers.get("content")
                 hash.rmContainers.get("content")
               );
               );
             }
             }
+
+            let historyProvider = this.fallbackToJHS ? hash.jhsContainers : hash.tsContainers;
+            let fieldName =  this.fallbackToJHS ? "containerId" : "id";
+
+            // Getting aggregated containers from the selected history provider
             if (
             if (
-              hash.tsContainers.get("length") > 0 &&
-              hash.tsContainers.get("content")
+              historyProvider.get("length") > 0 &&
+              historyProvider.get("content")
             ) {
             ) {
-              let tscontainer = [];
-              hash.tsContainers.get("content").forEach(function(o) {
-                if(!containerIdArr.contains(o.id)) {
-                  tscontainer.push(o);
+              let historyContainers = [];
+              historyProvider.get("content").forEach(function(o) {
+                if(!containerIdArr[o[fieldName]])) {
+                  historyContainers.push(o);
                 }
                 }
               }.bind(this));
               }.bind(this));
               containers = (containers || []).concat(
               containers = (containers || []).concat(
-                tscontainer);
+                historyContainers);
             }
             }
+
             this.set("attemptContainerList", containers);
             this.set("attemptContainerList", containers);
             this.initializeSelect(".js-fetch-logs-containers");
             this.initializeSelect(".js-fetch-logs-containers");
             if (containerId) {
             if (containerId) {
@@ -201,28 +209,36 @@ export default Ember.Controller.extend({
   }),
   }),
 
 
   fetchContainersForAttemptId(attemptId) {
   fetchContainersForAttemptId(attemptId) {
-    return Ember.RSVP.hash({
-      rmContainers: this.store
-        .query("yarn-container", {
-          app_attempt_id: attemptId
-        })
-        .catch(function() {
-          return Ember.A();
-        }),
-      tsContainers: this.store
-        .query("yarn-timeline-container", {
-          app_attempt_id: attemptId
-        })
-        .catch(function() {
-          return Ember.A();
-        })
-    });
+    let request = {};
+
+    request["rmContainers"] = this.store
+      .query("yarn-container", {
+        app_attempt_id: attemptId
+      })
+      .catch(function(error) {
+        return Ember.A();
+      });
+
+    let historyProvider = this.fallbackToJHS ? "jhsContainers" : "tsContainers";
+    let historyQuery = this.fallbackToJHS ? "yarn-jhs-container" : "yarn-timeline-container";
+
+    request[historyProvider] = this.store
+      .query(historyQuery, {
+        app_attempt_id: attemptId
+      })
+      .catch(function(error) {
+        return Ember.A();
+      });
+
+    return Ember.RSVP.hash(request);
   },
   },
 
 
   fetchLogFilesForContainerId(containerId) {
   fetchLogFilesForContainerId(containerId) {
+    let queryName = this.fallbackToJHS ? "yarn-jhs-log" : "yarn-log";
+
     return Ember.RSVP.hash({
     return Ember.RSVP.hash({
       logs: this.store
       logs: this.store
-        .query("yarn-log", {
+        .query(queryName, {
           containerId: containerId
           containerId: containerId
         })
         })
         .catch(function() {
         .catch(function() {
@@ -232,8 +248,10 @@ export default Ember.Controller.extend({
   },
   },
 
 
   fetchContentForLogFile(id) {
   fetchContentForLogFile(id) {
+    let queryName = this.fallbackToJHS ? 'yarn-app-jhs-log' : 'yarn-app-log';
+
     return Ember.RSVP.hash({
     return Ember.RSVP.hash({
-      logs: this.store.findRecord('yarn-app-log', id)
+      logs: this.store.findRecord(queryName, id)
     });
     });
   },
   },
 
 
@@ -265,10 +283,24 @@ export default Ember.Controller.extend({
     return logAggregationStatus !== "SUCCEEDED";
     return logAggregationStatus !== "SUCCEEDED";
   }),
   }),
 
 
-  isTimelineUnHealthy: function() {
-      if (this.model && this.model.timelineHealth) {
-        return this.model.timelineHealth.get('isTimelineUnHealthy');
+  fallbackToJHS: function() {
+    // Let's fall back to JHS if ATS is not available, but JHS is.
+    return this.model &&
+      (!this.model.timelineHealth || this.model.timelineHealth.get('isTimelineUnHealthy')) &&
+      this.model.jhsHealth && this.model.jhsHealth.get('isJHSHealthy');
+  }.property('model.timelineHealth', 'model.isJHSHealthy'),
+
+  areJHSandATSUnhealthy: function() {
+    if (this.model && this.model.timelineHealth) {
+      if (!this.model.timelineHealth.get('isTimelineUnHealthy')) {
+        return false;
       }
       }
-      return true;
-    }.property('model.timelineHealth')
+    }
+    if (this.model && this.model.jhsHealth) {
+      if (this.model.jhsHealth.get('isJHSHealthy')) {
+        return false;
+      }
+    }
+    return true;
+  }.property('model.timelineHealth', 'model.isJHSHealthy')
 });
 });

+ 59 - 40
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js

@@ -20,31 +20,46 @@
 
 
 import Ember from 'ember';
 import Ember from 'ember';
 
 
-function getYarnHttpProtocolScheme(rmhost, application) {
+function getConfigFromYarn(rmhost, application, config) {
   var httpUrl = window.location.protocol + '//' +
   var httpUrl = window.location.protocol + '//' +
     (ENV.hosts.localBaseAddress? ENV.hosts.localBaseAddress + '/' : '') + rmhost;
     (ENV.hosts.localBaseAddress? ENV.hosts.localBaseAddress + '/' : '') + rmhost;
 
 
-  httpUrl += '/conf?name=yarn.http.policy';
-  Ember.Logger.log("yarn.http.policy URL is: " + httpUrl);
+  httpUrl += '/conf?name=' + config;
+  Ember.Logger.log("The RM URL is: " + httpUrl);
 
 
-  var protocolScheme = "";
-  $.ajax({
-    type: 'GET',
-    dataType: 'json',
-    async: false,
-    context: this,
-    url: httpUrl,
-    success: function(data) {
-      protocolScheme = data.property.value;
-      Ember.Logger.log("Protocol scheme from RM: " + protocolScheme);
+  var configValue = "";
+    $.ajax({
+      type: 'GET',
+      dataType: 'json',
+      async: false,
+      context: this,
+      url: httpUrl,
+      success: function(data) {
+        configValue = data.property.value;
+        Ember.Logger.log("Value of the config returned from RM: " + configValue);
 
 
-      application.advanceReadiness();
-    },
-    error: function() {
-      application.advanceReadiness();
-    }
-  });
-  return protocolScheme;
+        application.advanceReadiness();
+      },
+      error: function() {
+        application.advanceReadiness();
+      }
+    });
+  return configValue;
+}
+
+function getJHSURL(rmhost, application, isHttpsSchemeEnabled) {
+  Ember.Logger.log("getJHSURL, params:rmhost=" + rmhost + ",application=" + application + ",isHttpsSchemeEnabled=" + isHttpsSchemeEnabled);
+  var config = '';
+  if (isHttpsSchemeEnabled) {
+    config = 'mapreduce.jobhistory.webapp.https.address';
+  } else {
+    config = 'mapreduce.jobhistory.webapp.address';
+  }
+  return getConfigFromYarn(rmhost, application, config);
+}
+
+function getYarnHttpProtocolScheme(rmhost, application) {
+  return getConfigFromYarn(rmhost, application, 'yarn.http.policy');
 }
 }
 
 
 function getTimeLineURL(rmhost, isHttpsSchemeEnabled) {
 function getTimeLineURL(rmhost, isHttpsSchemeEnabled) {
@@ -134,6 +149,21 @@ function getNodeManagerPort(rmhost, application) {
   return port;
   return port;
 }
 }
 
 
+function transformURL(url, hostname) {
+  // Deleting the scheme from the beginning of the url
+  url = url.replace(/(^\w+:|^)\/\//, '');
+
+  var address = url.split(":")[0];
+  var port = url.split(":")[1];
+  // Instead of localhost, use the name of the host
+  if (address === "0.0.0.0" || address === "localhost") {
+    url = hostname + ":" + port;
+  }
+
+  Ember.Logger.log("The transformed URL is: " + url);
+  return url;
+}
+
 function updateConfigs(application) {
 function updateConfigs(application) {
   var hostname = window.location.hostname;
   var hostname = window.location.hostname;
   var rmhost = hostname + (window.location.port ? ':' + window.location.port: '') +
   var rmhost = hostname + (window.location.port ? ':' + window.location.port: '') +
@@ -162,6 +192,13 @@ function updateConfigs(application) {
   Ember.Logger.log("NodeMananger port: " + nodeManagerPort);
   Ember.Logger.log("NodeMananger port: " + nodeManagerPort);
   ENV.nodeManagerPort = nodeManagerPort;
   ENV.nodeManagerPort = nodeManagerPort;
 
 
+  if (!ENV.hosts.jhsAddress) {
+    var jhsAddress = getJHSURL(rmhost, application, isHttpsSchemeEnabled);
+    jhsAddress = transformURL(jhsAddress, hostname);
+    Ember.Logger.log("The JHS address is " + jhsAddress);
+    ENV.hosts.jhsAddress = jhsAddress;
+  }
+
   if(!ENV.hosts.timelineWebAddress) {
   if(!ENV.hosts.timelineWebAddress) {
     var timelinehost = "";
     var timelinehost = "";
     $.ajax({
     $.ajax({
@@ -172,19 +209,10 @@ function updateConfigs(application) {
       url: getTimeLineURL(rmhost, isHttpsSchemeEnabled),
       url: getTimeLineURL(rmhost, isHttpsSchemeEnabled),
       success: function(data) {
       success: function(data) {
         timelinehost = data.property.value;
         timelinehost = data.property.value;
-        timelinehost = timelinehost.replace(/(^\w+:|^)\/\//, '');
+        timelinehost = transformURL(timelinehost, hostname);
         ENV.hosts.timelineWebAddress = timelinehost;
         ENV.hosts.timelineWebAddress = timelinehost;
-
-        var address = timelinehost.split(":")[0];
-        var port = timelinehost.split(":")[1];
-
         Ember.Logger.log("Timeline Address from RM: " + timelinehost);
         Ember.Logger.log("Timeline Address from RM: " + timelinehost);
 
 
-        if(address === "0.0.0.0" || address === "localhost") {
-          var updatedAddress =  hostname + ":" + port;
-          ENV.hosts.timelineWebAddress = updatedAddress;
-          Ember.Logger.log("Timeline Updated Address: " + updatedAddress);
-        }
         application.advanceReadiness();
         application.advanceReadiness();
       },
       },
       error: function() {
       error: function() {
@@ -206,19 +234,10 @@ function updateConfigs(application) {
       url: getTimeLineV1URL(rmhost, isHttpsSchemeEnabled),
       url: getTimeLineV1URL(rmhost, isHttpsSchemeEnabled),
       success: function(data) {
       success: function(data) {
         timelinehost = data.property.value;
         timelinehost = data.property.value;
-        timelinehost = timelinehost.replace(/(^\w+:|^)\/\//, '');
+        timelinehost = transformURL(timelinehost, hostname);
         ENV.hosts.timelineV1WebAddress = timelinehost;
         ENV.hosts.timelineV1WebAddress = timelinehost;
-
-        var address = timelinehost.split(":")[0];
-        var port = timelinehost.split(":")[1];
-
         Ember.Logger.log("Timeline V1 Address from RM: " + timelinehost);
         Ember.Logger.log("Timeline V1 Address from RM: " + timelinehost);
 
 
-        if(address === "0.0.0.0" || address === "localhost") {
-          var updatedAddress =  hostname + ":" + port;
-          ENV.hosts.timelineV1WebAddress = updatedAddress;
-          Ember.Logger.log("Timeline V1 Updated Address: " + updatedAddress);
-        }
         application.advanceReadiness();
         application.advanceReadiness();
       },
       },
       error: function() {
       error: function() {

+ 28 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/jhs-health.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';
+import Ember from 'ember';
+
+export default DS.Model.extend({
+  startedOn: DS.attr('string'),
+
+  isJHSHealthy: function() {
+    return this.get('startedOn') !== null;
+  }.property('startedOn')
+});

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

@@ -0,0 +1,25 @@
+/**
+ * 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'),
+  containerID: DS.attr('string'),
+  logFileName: DS.attr('string')
+});

+ 27 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-jhs-container.js

@@ -0,0 +1,27 @@
+/**
+ * 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.Model.extend({
+  containerLogInfo: DS.attr('array'),
+  logAggregationType: DS.attr('string'),
+  containerId: DS.attr('string'),
+  nodeId: DS.attr('string')
+});

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

@@ -0,0 +1,27 @@
+/**
+ * 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'),
+  fileSize: DS.attr('string'),
+  lastModifiedTime: DS.attr('string'),
+  containerId: DS.attr('string'),
+  nodeId: DS.attr('string')
+});

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/application.js

@@ -28,6 +28,9 @@ export default AbstractRoute.extend({
       userInfo: this.store.findAll('cluster-user-info', {reload: true}).catch(function() {
       userInfo: this.store.findAll('cluster-user-info', {reload: true}).catch(function() {
         return null;
         return null;
       }),
       }),
+      jhsHealth: this.store.queryRecord('jhs-health', {}).catch(function() {
+        return null;
+      }),
       timelineHealth: this.store.queryRecord('timeline-health', {}).catch(function() {
       timelineHealth: this.store.queryRecord('timeline-health', {}).catch(function() {
         return null;
         return null;
       })
       })
@@ -60,5 +63,6 @@ export default AbstractRoute.extend({
     this.store.unloadAll('ClusterInfo');
     this.store.unloadAll('ClusterInfo');
     this.store.unloadAll('cluster-user-info');
     this.store.unloadAll('cluster-user-info');
     this.store.unloadAll('timeline-health');
     this.store.unloadAll('timeline-health');
+    this.store.unloadAll('jhs-health');
   },
   },
 });
 });

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app/logs.js

@@ -32,6 +32,11 @@ export default AbstractRoute.extend(AppAttemptMixin, {
         return [];
         return [];
       }),
       }),
       app: this.fetchAppInfoFromRMorATS(app_id, this.store),
       app: this.fetchAppInfoFromRMorATS(app_id, this.store),
+      jhsHealth: this.store.queryRecord('jhs-health', {}).catch(function(error) {
+        Ember.Logger.log("jhs-health querying failed");
+        Ember.Logger.log(error);
+        return null;
+      }),
       timelineHealth: this.store.queryRecord('timeline-health', {}).catch(function() {
       timelineHealth: this.store.queryRecord('timeline-health', {}).catch(function() {
         return null;
         return null;
       })
       })
@@ -55,6 +60,7 @@ export default AbstractRoute.extend(AppAttemptMixin, {
     this.store.unloadAll('yarn-timeline-appattempt');
     this.store.unloadAll('yarn-timeline-appattempt');
     this.store.unloadAll('yarn-container');
     this.store.unloadAll('yarn-container');
     this.store.unloadAll('yarn-timeline-container');
     this.store.unloadAll('yarn-timeline-container');
+    this.store.unloadAll('yarn-jhs-container');
     this.store.unloadAll('yarn-log');
     this.store.unloadAll('yarn-log');
     if (this.controller) {
     if (this.controller) {
       this.controller.resetAfterRefresh();
       this.controller.resetAfterRefresh();

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

@@ -0,0 +1,32 @@
+/**
+ * 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 Ember from 'ember';
+
+export default DS.JSONAPISerializer.extend({
+  normalizeSingleResponse(store, primaryModelClass, payload) {
+    var fixedPayload = {
+      id: Date.now(),
+      type: primaryModelClass.modelName,
+      attributes: payload.historyInfo
+    };
+
+    return { data: fixedPayload };
+  }
+});

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

@@ -0,0 +1,38 @@
+/**
+ * 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*/) {
+    // Convert plain text response into JSON.
+    // ID is of the form containerId!fileName
+    var splits = Converter.splitForAppLogs(id);
+    var convertedPayload = {
+      id: id,
+      type: primaryModelClass.modelName,
+      attributes: {
+        logs: payload,
+        containerID: splits[0],
+        logFileName: splits[1]
+      }
+    };
+    return { data: convertedPayload };
+  },
+});

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

@@ -0,0 +1,62 @@
+/**
+ * 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 Ember from 'ember';
+import Converter from 'yarn-ui/utils/converter';
+
+export default DS.JSONAPISerializer.extend({
+  internalNormalizeSingleResponse(store, primaryModelClass, payload) {
+
+    var fixedPayload = {
+      id: payload.containerId,
+      type: primaryModelClass.modelName,
+      attributes: {
+        containerLogInfo: payload.containerLogInfo,
+        logAggregationType: payload.logAggregationType,
+        containerId: payload.containerId,
+        nodeId: payload.nodeId
+      }
+    };
+    return fixedPayload;
+  },
+
+  normalizeSingleResponse(store, primaryModelClass, payload/*, id, requestType*/) {
+    var normalized = this.internalNormalizeSingleResponse(store,
+        primaryModelClass, payload);
+    return {
+      data: normalized
+    };
+  },
+
+  normalizeArrayResponse(store, primaryModelClass, payload/*, id, requestType*/) {
+
+    payload = payload["containerLogsInfo"]
+
+    var normalizedArrayResponse = {
+      data: []
+    };
+    if (payload && Ember.isArray(payload) && !Ember.isEmpty(payload)) {
+      normalizedArrayResponse.data = payload.map(singleContainer => {
+        return this.internalNormalizeSingleResponse(store, primaryModelClass,
+          singleContainer);
+      });
+    }
+    return normalizedArrayResponse;
+  }
+});

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

@@ -0,0 +1,56 @@
+/**
+ * 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 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;
+  }
+});

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/services/hosts.js

@@ -80,6 +80,10 @@ export default Ember.Service.extend({
     return this.normalizeURL(this.get("env.app.hosts.rmWebAddress"));
     return this.normalizeURL(this.get("env.app.hosts.rmWebAddress"));
   }),
   }),
 
 
+  jhsAddress: Ember.computed(function() {
+    return this.normalizeURL(this.get("env.app.hosts.jhsAddress"));
+  }),
+
   dashWebAddress: Ember.computed(function () {
   dashWebAddress: Ember.computed(function () {
     return this.normalizeURL(this.get("env.app.hosts.dashWebAddress"));
     return this.normalizeURL(this.get("env.app.hosts.dashWebAddress"));
   }),
   }),

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app/logs.hbs

@@ -32,11 +32,11 @@
             <img src="assets/images/spinner.gif" alt="Loading...">
             <img src="assets/images/spinner.gif" alt="Loading...">
           </div>
           </div>
         {{/if}}
         {{/if}}
-        {{#if isTimelineUnHealthy}}
+        {{#if areJHSandATSUnhealthy}}
           <div class="row">
           <div class="row">
             <div class="col-md-8 col-md-offset-2 alert alert-warning text-center">
             <div class="col-md-8 col-md-offset-2 alert alert-warning text-center">
               <span class="glyphicon glyphicon-warning-sign" style="padding-right: 10px"></span>
               <span class="glyphicon glyphicon-warning-sign" style="padding-right: 10px"></span>
-              <span>Logs are unavailable because Application Timeline Service seems unhealthy.</span>
+              <span>Logs are unavailable because Application Timeline Service seems unhealthy and could not connect to the JobHistory server.</span>
             </div>
             </div>
           </div>
           </div>
         {{else}}
         {{else}}

+ 7 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/config/configs.env

@@ -43,6 +43,12 @@ ENV = {
     /*
     /*
      * Protocol scheme. It can be "http:" or "https:". By default, http is used.
      * Protocol scheme. It can be "http:" or "https:". By default, http is used.
      */
      */
-      //protocolScheme: "http:"
+      //protocolScheme: "http:",
+
+    /*
+     * JHS web interface can be configured below.
+     */
+      //jhsAddress: "localhost:19888"
+
   },
   },
 };
 };

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/config/default-config.js

@@ -22,6 +22,7 @@ module.exports = { // YARN UI App configurations
       timelineWebAddress: "localhost:8188",
       timelineWebAddress: "localhost:8188",
       timelineV1WebAddress: "localhost:8188",
       timelineV1WebAddress: "localhost:8188",
       rmWebAddress: "localhost:8088",
       rmWebAddress: "localhost:8088",
+      jhsAddress: "localhost:19888",
       protocolScheme: "http:",
       protocolScheme: "http:",
       isSecurityEnabled: ""
       isSecurityEnabled: ""
     },
     },
@@ -33,6 +34,7 @@ module.exports = { // YARN UI App configurations
       timelineV2: 'ws/v2/timeline',
       timelineV2: 'ws/v2/timeline',
       timelineV2Log: 'ws/v2/applicationlog',
       timelineV2Log: 'ws/v2/applicationlog',
       dashService: 'app/v1/services',
       dashService: 'app/v1/services',
-      node: '{nodeAddress}/ws/v1/node'
+      node: '{nodeAddress}/ws/v1/node',
+      jhs: 'ws/v1/history'
     },
     },
 };
 };