Bladeren bron

HDFS-12005. Ozone: Web interface for SCM. Contributed by Elek, Marton.

Anu Engineer 8 jaren geleden
bovenliggende
commit
dcc21a4f93
21 gewijzigde bestanden met toevoegingen van 557 en 291 verwijderingen
  1. 37 27
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/ServiceRuntimeInfo.java
  2. 32 30
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/ServiceRuntimeInfoImpl.java
  3. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KSMMXBean.java
  4. 10 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java
  5. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/SCMMXBean.java
  6. 12 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java
  7. 16 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java
  8. 30 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockmanagerMXBean.java
  9. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/OzoneHttpServer.java
  10. 2 26
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/ksm/index.html
  11. 2 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/ksm/ksm-metrics.html
  12. 10 161
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/ksm/ksm.js
  13. 19 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/scm/index.html
  14. 60 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/scm/scm-overview.html
  15. 41 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/scm/scm.js
  16. 234 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/ozone.js
  17. 0 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/templates/jvm.html
  18. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/templates/overview.html
  19. 1 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/templates/rpc-metrics.html
  20. 38 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/templates/tools.html
  21. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java

+ 37 - 27
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/ServiceRuntimeInfo.java

@@ -17,38 +17,48 @@
 
 package org.apache.hadoop.jmx;
 
-import org.apache.hadoop.util.VersionInfo;
-
 /**
- * Helper base class to report the standard version and runtime information.
+ * Common runtime information for any service components.
+ *
+ * Note: it's intentional to not use MXBean or MBean as a suffix  of the name.
+ *
+ * Most of the services extends the ServiceRuntimeInfoImpl class and also
+ * implements a specific MXBean interface which extends this interface.
+ *
+ * This inheritance from multiple path could confuse the jmx system and
+ * some jmx properties could be disappeared.
+ *
+ * The solution is to always extend this interface and use the jmx naming
+ * convention in the new interface..
  */
-public class ServiceRuntimeInfo implements ServiceRuntimeInfoMBean {
-
-  private long startedTimeInMillis;
-
-  @Override
-  public String getVersion() {
-    return VersionInfo.getVersion() + ", r" + VersionInfo.getRevision();
-  }
+public interface ServiceRuntimeInfo {
 
-  @Override
-  public String getSoftwareVersion() {
-    return VersionInfo.getVersion();
-  }
+  /**
+   * Gets the version of Hadoop.
+   *
+   * @return the version
+   */
+  String getVersion();
 
-  @Override
-  public String getCompileInfo() {
-    return VersionInfo.getDate() + " by " + VersionInfo.getUser() + " from "
-        + VersionInfo.getBranch();
-  }
+  /**
+   * Get the version of software running on the Namenode
+   *
+   * @return a string representing the version
+   */
+  String getSoftwareVersion();
 
-  @Override
-  public long getStartedTimeInMillis() {
-    return System.currentTimeMillis() - startedTimeInMillis;
-  }
+  /**
+   * Get the compilation information which contains date, user and branch
+   *
+   * @return the compilation information, as a JSON string.
+   */
+  String getCompileInfo();
 
-  public void setStartTime() {
-    startedTimeInMillis = System.currentTimeMillis();
-  }
+  /**
+   * Gets the NN start time in milliseconds.
+   *
+   * @return the NN start time in msec
+   */
+  long getStartedTimeInMillis();
 
 }

+ 32 - 30
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/ServiceRuntimeInfoMBean.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/ServiceRuntimeInfoImpl.java

@@ -17,37 +17,39 @@
 
 package org.apache.hadoop.jmx;
 
+import org.apache.hadoop.util.VersionInfo;
+
 /**
- * Common runtime information for any service components.
+ * Helper base class to report the standard version and runtime information.
+ *
  */
-public interface ServiceRuntimeInfoMBean {
-
-  /**
-   * Gets the version of Hadoop.
-   *
-   * @return the version
-   */
-  String getVersion();
-
-  /**
-   * Get the version of software running on the Namenode
-   *
-   * @return a string representing the version
-   */
-  String getSoftwareVersion();
-
-  /**
-   * Get the compilation information which contains date, user and branch
-   *
-   * @return the compilation information, as a JSON string.
-   */
-  String getCompileInfo();
-
-  /**
-   * Gets the NN start time in milliseconds.
-   *
-   * @return the NN start time in msec
-   */
-  long getStartedTimeInMillis();
+public class ServiceRuntimeInfoImpl implements ServiceRuntimeInfo {
+
+  private long startedTimeInMillis;
+
+  @Override
+  public String getVersion() {
+    return VersionInfo.getVersion() + ", r" + VersionInfo.getRevision();
+  }
+
+  @Override
+  public String getSoftwareVersion() {
+    return VersionInfo.getVersion();
+  }
+
+  @Override
+  public String getCompileInfo() {
+    return VersionInfo.getDate() + " by " + VersionInfo.getUser() + " from "
+        + VersionInfo.getBranch();
+  }
+
+  @Override
+  public long getStartedTimeInMillis() {
+    return startedTimeInMillis;
+  }
+
+  public void setStartTime() {
+    startedTimeInMillis = System.currentTimeMillis();
+  }
 
 }

+ 2 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KSMMXBean.java

@@ -19,15 +19,13 @@
 package org.apache.hadoop.ozone.ksm;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.jmx.ServiceRuntimeInfoMBean;
-
-import java.util.Map;
+import org.apache.hadoop.jmx.ServiceRuntimeInfo;
 
 /**
  * This is the JMX management interface for ksm information.
  */
 @InterfaceAudience.Private
-public interface KSMMXBean extends ServiceRuntimeInfoMBean {
+public interface KSMMXBean extends ServiceRuntimeInfo {
 
   String getRpcPort();
 }

+ 10 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java

@@ -23,7 +23,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.jmx.ServiceRuntimeInfo;
+import org.apache.hadoop.jmx.ServiceRuntimeInfoImpl;
 import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
 import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
@@ -52,7 +52,9 @@ import org.slf4j.LoggerFactory;
 import javax.management.ObjectName;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import static org.apache.hadoop.ozone.ksm.KSMConfigKeys
     .OZONE_KSM_ADDRESS_KEY;
@@ -69,7 +71,7 @@ import static org.apache.hadoop.util.ExitUtil.terminate;
  * Ozone Keyspace manager is the metadata manager of ozone.
  */
 @InterfaceAudience.LimitedPrivate({"HDFS", "CBLOCK", "OZONE", "HBASE"})
-public class KeySpaceManager extends ServiceRuntimeInfo
+public class KeySpaceManager extends ServiceRuntimeInfoImpl
     implements KeySpaceManagerProtocol, KSMMXBean {
   private static final Logger LOG =
       LoggerFactory.getLogger(KeySpaceManager.class);
@@ -557,8 +559,13 @@ public class KeySpaceManager extends ServiceRuntimeInfo
   }
 
   private void registerMXBean() {
+    Map<String, String> jmxProperties = new HashMap<String, String>();
+    jmxProperties.put("component", "ServerRuntime");
     this.ksmInfoBeanName =
-        MBeans.register("KeySpaceManager", "KeySpaceManagerInfo", this);
+        MBeans.register("KeySpaceManager",
+            "KeySpaceManagerInfo",
+            jmxProperties,
+            this);
   }
 
   private void unregisterMXBean() {

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/SCMMXBean.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.scm;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.jmx.ServiceRuntimeInfo;
 
 import java.util.Map;
 
@@ -27,7 +28,7 @@ import java.util.Map;
  * This is the JMX management interface for scm information.
  */
 @InterfaceAudience.Private
-public interface SCMMXBean {
+public interface SCMMXBean extends ServiceRuntimeInfo {
 
   /**
    * Get the number of data nodes that in all states.

+ 12 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/StorageContainerManager.java

@@ -27,6 +27,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.jmx.ServiceRuntimeInfoImpl;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -132,7 +134,7 @@ import static org.apache.hadoop.util.ExitUtil.terminate;
  * datanodes and create a container, which then can be used to store data.
  */
 @InterfaceAudience.LimitedPrivate({"HDFS", "CBLOCK", "OZONE", "HBASE"})
-public class StorageContainerManager
+public class StorageContainerManager extends ServiceRuntimeInfoImpl
     implements StorageContainerDatanodeProtocol,
     StorageContainerLocationProtocol, ScmBlockLocationProtocol, SCMMXBean {
 
@@ -293,8 +295,13 @@ public class StorageContainerManager
   }
 
   private void registerMXBean() {
-    this.scmInfoBeanName = MBeans.register("StorageContainerManager",
-        "StorageContainerManagerInfo", this);
+    Map<String, String> jmxProperties = new HashMap<>();
+    jmxProperties.put("component", "ServerRuntime");
+    this.scmInfoBeanName =
+        MBeans.register("StorageContainerManager",
+            "StorageContainerManagerInfo",
+            jmxProperties,
+            this);
   }
 
   private void unregisterMXBean() {
@@ -564,6 +571,7 @@ public class StorageContainerManager
   public void start() throws IOException {
     LOG.info(buildRpcServerStartMessage(
         "StorageContainerLocationProtocol RPC server", clientRpcAddress));
+    DefaultMetricsSystem.initialize("StorageContainerManager");
     clientRpcServer.start();
     LOG.info(buildRpcServerStartMessage(
         "ScmBlockLocationProtocol RPC server", blockRpcAddress));
@@ -571,9 +579,9 @@ public class StorageContainerManager
     LOG.info(buildRpcServerStartMessage("RPC server for DataNodes",
         datanodeRpcAddress));
     datanodeRpcServer.start();
-
     httpServer.start();
 
+    setStartTime();
 
   }
 

+ 16 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockManagerImpl.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.scm.block;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.scm.container.Mapping;
 import org.apache.hadoop.ozone.scm.exceptions.SCMException;
@@ -35,13 +36,14 @@ import org.apache.hadoop.utils.MetadataStoreBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.management.ObjectName;
 import java.io.File;
 import java.io.IOException;
 
 import java.util.ArrayList;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -68,7 +70,7 @@ import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
 /**
  * Block Manager manages the block access for SCM.
  */
-public class BlockManagerImpl implements BlockManager {
+public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
   private static final Logger LOG =
       LoggerFactory.getLogger(BlockManagerImpl.class);
 
@@ -84,6 +86,8 @@ public class BlockManagerImpl implements BlockManager {
   private Map<String, Long> openContainers;
   private final int containerProvisionBatchSize;
   private final Random rand;
+  private final ObjectName mxBean;
+
 
   /**
    * Constructor.
@@ -122,7 +126,7 @@ public class BlockManagerImpl implements BlockManager {
         .setCacheSize(this.cacheSize * OzoneConsts.MB)
         .build();
 
-    openContainers = new HashMap<>();
+    openContainers = new ConcurrentHashMap<>();
     loadOpenContainers();
 
     this.containerProvisionBatchSize = conf.getInt(
@@ -130,6 +134,8 @@ public class BlockManagerImpl implements BlockManager {
         ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT);
     rand = new Random();
     this.lock = new ReentrantLock();
+
+    mxBean = MBeans.register("BlockManager", "BlockManagerImpl", this);
   }
 
   // TODO: close full (or almost full) containers with a separate thread.
@@ -361,5 +367,12 @@ public class BlockManagerImpl implements BlockManager {
     if (openContainerStore != null) {
       openContainerStore.close();
     }
+
+    MBeans.unregister(mxBean);
+  }
+
+  @Override
+  public int getOpenContainersNo() {
+    return openContainers.size();
   }
 }

+ 30 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/scm/block/BlockmanagerMXBean.java

@@ -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.
+ */
+
+package org.apache.hadoop.ozone.scm.block;
+
+
+/**
+ * JMX interface for the block manager.
+ */
+public interface BlockmanagerMXBean {
+
+  /**
+   * Number of open containers manager by the block manager.
+   */
+  int getOpenContainersNo();
+}

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/OzoneHttpServer.java

@@ -138,8 +138,8 @@ public abstract class OzoneHttpServer {
       String realAddress = NetUtils.getHostPortString(httpAddress);
       conf.set(getHttpAddressKey(), realAddress);
       LOG.info(
-          String.format("HTTP server of SCM is listening at http://%s",
-              realAddress));
+          String.format("HTTP server of %s is listening at http://%s",
+              name.toUpperCase(), realAddress));
     }
 
     if (policy.isHttpsEnabled()) {
@@ -147,8 +147,8 @@ public abstract class OzoneHttpServer {
       String realAddress = NetUtils.getHostPortString(httpsAddress);
       conf.set(getHttpsAddressKey(), realAddress);
       LOG.info(
-          String.format("HTTP server of SCM is listening at https://%s",
-              realAddress));
+          String.format("HTTP server of %s is listening at https://%s",
+              name.toUpperCase(), realAddress));
     }
   }
 

+ 2 - 26
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/ksm/index.html

@@ -48,32 +48,7 @@
             </button>
             <a class="navbar-brand" href="#">HDFS KSM</a>
         </div>
-        <div id="navbar" class="collapse navbar-collapse">
-            <ul class="nav navbar-nav" id="ui-tabs">
-                <li>
-                    <a class="dropdown-toggle" id="toolsMenu"
-                       data-toggle="dropdown" aria-haspopup="true"
-                       aria-expanded="true" href="#">
-                        External tools
-                        <span class="caret"></span>
-                    </a>
-                    <ul class="dropdown-menu" aria-labelledby="toolsMenu">
-                        <li><a href="/jmx">JMX <span
-                                class="glyphicon glyphicon-new-window"
-                                aria-hidden="true"></span></a></li>
-                        <li><a href="/conf">Config <span
-                                class="glyphicon glyphicon-new-window"
-                                aria-hidden="true"></a></li>
-                        <li><a href="/stacks">Stacks <span
-                                class="glyphicon glyphicon-new-window"
-                                aria-hidden="true"></a></li>
-                        <li><a href="/logLevel">Log levels <span
-                                class="glyphicon glyphicon-new-window"
-                                aria-hidden="true"></a></li>
-                    </ul>
-                </li>
-            </ul>
-        </div><!--/.nav-collapse -->
+        <common-tools></common-tools>
     </div>
 </header>
 
@@ -92,6 +67,7 @@
 <script src="/static/d3-3.5.17.min.js"></script>
 <script src="/static/nvd3-1.8.5.min.js"></script>
 <script src="/static/angular-nvd3-1.0.9.min.js"></script>
+<script src="/static/ozone.js"></script>
 <script src="/ksm.js"></script>
 <script src="/static/bootstrap-3.0.2/js/bootstrap.min.js"></script>
 </body>

+ 2 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/ksm/ksm-metrics.html

@@ -18,12 +18,9 @@
 
 <div ng-repeat="(type,numbers) in $ctrl.metrics.nums">
     <h2>{{type}}</h2>
-    <div class="alert alert-info" ng-hide="$ctrl.metrics.nums > 0">
-        No {{type}} related requests, yet.
-    </div>
-    <div class="container" ng-show="$ctrl.metrics.nums > 0">
+    <div class="container">
         <div class="col-md-6">
-            <h3>Requests</h3>
+            <h3>Requests ({{numbers.ops}} ops)</h3>
             <nvd3 options="$ctrl.graphOptions"
                   data="numbers.all"></nvd3>
         </div>

+ 10 - 161
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/ksm/ksm.js

@@ -21,35 +21,9 @@
     var isIgnoredJmxKeys = function (key) {
         return key == 'name' || key == 'modelerType' || key.match(/tag.*/);
     };
-    angular.module('ksm', ['nvd3'])
-    angular.module('ksm').component('overview', {
-        templateUrl: 'overview.html',
-        controller: function ($http) {
-            var ctrl = this;
-            $http.get("/jmx?qry=Hadoop:service=KeySpaceManager,name=KeySpaceManagerInfo")
-                .then(function (result) {
-                    ctrl.jmx = result.data.beans[0]
-                })
-        }
-    });
-    angular.module('ksm').component('jvmParameters', {
-        templateUrl: 'jvm.html',
-        controller: function ($http) {
-            var ctrl = this
-            $http.get("/jmx?qry=java.lang:type=Runtime")
-                .then(function (result) {
-                    ctrl.jmx = result.data.beans[0];
 
-                    //convert array to a map
-                    var systemProperties = {}
-                    for (var idx in ctrl.jmx.SystemProperties) {
-                        var item = ctrl.jmx.SystemProperties[idx];
-                        systemProperties[item.key.replace(/\./g,"_")] = item.value;
-                    }
-                    ctrl.jmx.SystemProperties = systemProperties;
-                })
-        }
-    });
+    angular.module('ksm', ['ozone', 'nvd3']);
+
     angular.module('ksm').component('ksmMetrics', {
         templateUrl: 'ksm-metrics.html',
         controller: function ($http) {
@@ -103,10 +77,14 @@
                                     value: metrics[key]
                                 })
                             } else {
-                                groupedMetrics.nums[type].all.push({
-                                    key: name,
-                                    value: metrics[key]
-                                })
+                                if (name == "Ops") {
+                                    groupedMetrics.nums[type].ops = metrics[key]
+                                } else {
+                                    groupedMetrics.nums[type].all.push({
+                                        key: name,
+                                        value: metrics[key]
+                                    })
+                                }
                             }
                         } else if (isIgnoredJmxKeys(key)) {
                             //ignore
@@ -121,134 +99,5 @@
                 })
         }
     });
-    angular.module('ksm').component('rpcMetrics', {
-        template: '<div ng-repeat="metric in $ctrl.metrics"><rpc-metric jmxdata="metric"></rpc-metric></div>',
-        controller: function ($http) {
-            var ctrl = this;
-            $http.get("/jmx?qry=Hadoop:service=KeySpaceManager,name=RpcActivityForPort*")
-                .then(function (result) {
-                    ctrl.metrics = result.data.beans;
-                })
-        }
-    });
-    angular.module('ksm').component('rpcMetric', {
-        bindings: {
-            jmxdata: '<'
-        },
-        templateUrl: 'rpc-metrics.html',
-        controller: function () {
-            var ctrl = this;
-
-
-            ctrl.percentileGraphOptions = {
-                chart: {
-                    type: 'discreteBarChart',
-                    height: 450,
-                    margin: {
-                        top: 20,
-                        right: 20,
-                        bottom: 50,
-                        left: 55
-                    },
-                    x: function (d) {
-                        return d.label;
-                    },
-                    y: function (d) {
-                        return d.value;
-                    },
-                    showValues: true,
-                    valueFormat: function (d) {
-                        return d3.format(',.1f')(d);
-                    },
-                    duration: 500,
-                    xAxis: {
-                        axisLabel: 'Percentage'
-                    },
-                    yAxis: {
-                        axisLabel: 'Latency (ms)',
-                        axisLabelDistance: -10
-                    }
-                }
-            };
-
-            ctrl.$onChanges = function (data) {
-                var groupedMetrics = {}
-
-                var createPercentageMetrics = function (metricName, window) {
-                    groupedMetrics.percentiles = groupedMetrics['percentiles'] || {}
-                    groupedMetrics.percentiles[metricName] = groupedMetrics.percentiles[metricName] || {};
-                    groupedMetrics.percentiles[metricName][window] = groupedMetrics.percentiles[metricName][window] || {
-                            graphdata: [{
-                                key: window,
-                                values: []
-                            }], numOps: 0
-                        };
-
-                };
-                var metrics = ctrl.jmxdata;
-                for (var key in metrics) {
-                    var percentile = key.match(/(.*Time)(\d+s)(\d+th)PercentileLatency/);
-                    var percentileNumOps = key.match(/(.*Time)(\d+s)NumOps/);
-                    var successFailures = key.match(/(.*)(Success|Failures)/);
-                    var numAverages = key.match(/(.*Time)(NumOps|AvgTime)/);
-                    if (percentile) {
-                        var metricName = percentile[1];
-                        var window = percentile[2];
-                        var percentage = percentile[3]
-                        createPercentageMetrics(metricName, window);
-
-
-                        groupedMetrics.percentiles[metricName][window].graphdata[0]
-                            .values.push({
-                            label: percentage,
-                            value: metrics[key]
-                        })
-                    } else if (successFailures) {
-                        var metricName = successFailures[1];
-                        groupedMetrics.successfailures = groupedMetrics['successfailures'] || {}
-                        groupedMetrics.successfailures[metricName] = groupedMetrics.successfailures[metricName] || {
-                                success: 0,
-                                failures: 0
-                            };
-                        if (successFailures[2] == 'Success') {
-                            groupedMetrics.successfailures[metricName].success = metrics[key];
-                        } else {
-                            groupedMetrics.successfailures[metricName].failures = metrics[key];
-                        }
-
-                    } else if (numAverages) {
-                        var metricName = numAverages[1];
-                        groupedMetrics.numavgs = groupedMetrics['numavgs'] || {}
-                        groupedMetrics.numavgs[metricName] = groupedMetrics.numavgs[metricName] || {
-                                numOps: 0,
-                                avgTime: 0
-                            };
-                        if (numAverages[2] == 'NumOps') {
-                            groupedMetrics.numavgs[metricName].numOps = metrics[key];
-                        } else {
-                            groupedMetrics.numavgs[metricName].avgTime = metrics[key];
-                        }
-
-                    } else if (percentileNumOps) {
-                        var metricName = percentileNumOps[1];
-                        var window = percentileNumOps[2];
-                        createPercentageMetrics(metricName, window);
-                        groupedMetrics.percentiles[metricName][window].numOps = metrics[key];
-                    } else if (isIgnoredJmxKeys(key)) {
-                        //ignore
-                    } else {
-                        groupedMetrics.others = groupedMetrics.others || [];
-                        groupedMetrics.others.push({
-                            'key': key,
-                            'value': metrics[key]
-                        });
-                    }
-
-                }
-                ctrl.metrics = groupedMetrics;
-            };
-
-        }
-    });
 
 })();

+ 19 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/scm/index.html

@@ -28,14 +28,15 @@
 
     <link href="/static/bootstrap-3.0.2/css/bootstrap.min.css" rel="stylesheet">
     <link href="/static/hadoop.css" rel="stylesheet">
+    <link href="/static/nvd3-1.8.5.min.css" rel="stylesheet">
 
     <link href="/main.css" rel="stylesheet">
 
 </head>
 
-<body>
+<body ng-app="scm">
 
-<nav class="navbar navbar-inverse navbar-fixed-top">
+<header class="navbar navbar-inverse navbar-fixed-top bs-docs-nav">
     <div class="container">
         <div class="navbar-header">
             <button type="button" class="navbar-toggle collapsed" data-toggle="collapse" data-target="#navbar"
@@ -47,29 +48,29 @@
             </button>
             <a class="navbar-brand" href="#">HDFS SCM</a>
         </div>
-        <div id="navbar" class="collapse navbar-collapse">
-            <ul class="nav navbar-nav">
-                <li><a href="#">Home</a></li>
-                <li><a href="/jmx">JMX</a></li>
-                <li><a href="/conf">Config</a></li>
-                <li><a href="/stacks">Stacks</a></li>
-                <li><a href="/logLevel">Log levels</a></li>
-
-            </ul>
-        </div><!--/.nav-collapse -->
+        <common-tools></common-tools>
     </div>
-</nav>
+</header>
 
 <div class="container">
 
-    <div class="starter-template">
-        <h1>HDFS SCM</h1>
-    </div>
+    <overview>
+        <scm-overview>
+        </scm-overview>
+    </overview>
+    <rpc-metrics></rpc-metrics>
 
-</div><!-- /.container -->
 
+</div><!-- /.container -->
 
 <script src="/static/jquery-1.10.2.min.js"></script>
-<script src="/static/bootstrap-3.0.2//js/bootstrap.min.js"></script>
+<script src="/static/angular-1.6.4.min.js"></script>
+<script src="/static/angular-route-1.6.4.min.js"></script>
+<script src="/static/d3-3.5.17.min.js"></script>
+<script src="/static/nvd3-1.8.5.min.js"></script>
+<script src="/static/angular-nvd3-1.0.9.min.js"></script>
+<script src="/static/ozone.js"></script>
+<script src="/scm.js"></script>
+<script src="/static/bootstrap-3.0.2/js/bootstrap.min.js"></script>
 </body>
 </html>

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/scm/scm-overview.html

@@ -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.
+-->
+<h2>Node counts</h2>
+
+<table class="table table-bordered table-striped" class="col-md-6">
+    <tbody>
+    <tr ng-repeat="typestat in $ctrl.overview.jmx.NodeCount">
+        <td>{{typestat.key}}</td>
+        <td>{{typestat.value}}</td>
+    </tr>
+    </tbody>
+</table>
+
+<h2>Status</h2>
+<table class="table table-bordered table-striped" class="col-md-6">
+    <tbody>
+    <tr>
+        <td>Client Rpc port</td>
+        <td>{{$ctrl.overview.jmx.ClientRpcPort}}</td>
+    </tr>
+    <tr>
+        <td>Datanode Rpc port</td>
+        <td>{{$ctrl.overview.jmx.DatanodeRpcPort}}</td>
+    </tr>
+    <tr>
+        <td>Block Manager: Open containers</td>
+        <td>{{$ctrl.blockmanagermetrics.OpenContainersNo}}</td>
+    </tr>
+    <tr>
+        <td>Node Manager: Minimum chill mode nodes)</td>
+        <td>{{$ctrl.nodemanagermetrics.MinimumChillModeNodes}}</td>
+    </tr>
+    <tr>
+        <td>Node Manager: Out-of-node chill mode</td>
+        <td>{{$ctrl.nodemanagermetrics.OutOfNodeChillMode}}</td>
+    </tr>
+    <tr>
+        <td>Node Manager: Chill mode status</td>
+        <td>{{$ctrl.nodemanagermetrics.ChillModeStatus}}</td>
+    </tr>
+    <tr>
+        <td>Node Manager: Manual chill mode</td>
+        <td>{{$ctrl.nodemanagermetrics.InManualChillMode}}</td>
+    </tr>
+    </tbody>
+</table>

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/scm/scm.js

@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+(function () {
+    "use strict";
+    angular.module('scm', ['ozone', 'nvd3']);
+
+    angular.module('scm').component('scmOverview', {
+        templateUrl: 'scm-overview.html',
+        require: {
+            overview: "^overview"
+        },
+        controller: function ($http) {
+            var ctrl = this;
+            $http.get("/jmx?qry=Hadoop:service=BlockManager,name=*")
+                .then(function (result) {
+                    ctrl.blockmanagermetrics = result.data.beans[0];
+                });
+            $http.get("/jmx?qry=Hadoop:service=SCMNodeManager,name=SCMNodeManagerInfo")
+                .then(function (result) {
+                    ctrl.nodemanagermetrics = result.data.beans[0];
+                });
+
+        }
+    });
+
+})();

+ 234 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/ozone.js

@@ -0,0 +1,234 @@
+/**
+ * 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.
+ */
+(function () {
+    "use strict";
+
+    var isIgnoredJmxKeys = function (key) {
+        return key == 'name' || key == 'modelerType' || key.match(/tag.*/);
+    };
+    angular.module('ozone', ['nvd3'])
+    angular.module('ozone').component('overview', {
+        templateUrl: 'static/templates/overview.html',
+        transclude: true,
+        controller: function ($http) {
+            var ctrl = this;
+            $http.get("/jmx?qry=Hadoop:service=*,name=*,component=ServerRuntime")
+                .then(function (result) {
+                    ctrl.jmx = result.data.beans[0]
+                })
+        }
+    });
+    angular.module('ozone').component('jvmParameters', {
+        templateUrl: 'static/templates/jvm.html',
+        controller: function ($http) {
+            var ctrl = this
+            $http.get("/jmx?qry=java.lang:type=Runtime")
+                .then(function (result) {
+                    ctrl.jmx = result.data.beans[0];
+
+                    //convert array to a map
+                    var systemProperties = {}
+                    for (var idx in ctrl.jmx.SystemProperties) {
+                        var item = ctrl.jmx.SystemProperties[idx];
+                        systemProperties[item.key.replace(/\./g, "_")] = item.value;
+                    }
+                    ctrl.jmx.SystemProperties = systemProperties;
+                })
+        }
+    });
+
+    angular.module('ozone').component('rpcMetrics', {
+        template: '<h1>Rpc metrics</h1><tabs>' +
+        '<pane ng-repeat="metric in $ctrl.metrics" ' +
+        'title="Port {{metric[\'tag.port\']}}">' +
+        '<rpc-metric jmxdata="metric"></rpc-metric></pane>' +
+        '</tabs>',
+        controller: function ($http) {
+            var ctrl = this;
+            $http.get("/jmx?qry=Hadoop:service=*,name=RpcActivityForPort*")
+                .then(function (result) {
+                    ctrl.metrics = result.data.beans;
+                })
+        }
+    });
+    angular.module('ozone').component('rpcMetric', {
+        bindings: {
+            jmxdata: '<'
+        },
+        templateUrl: 'static/templates/rpc-metrics.html',
+        controller: function () {
+            var ctrl = this;
+
+
+            ctrl.percentileGraphOptions = {
+                chart: {
+                    type: 'discreteBarChart',
+                    height: 450,
+                    margin: {
+                        top: 20,
+                        right: 20,
+                        bottom: 50,
+                        left: 55
+                    },
+                    x: function (d) {
+                        return d.label;
+                    },
+                    y: function (d) {
+                        return d.value;
+                    },
+                    showValues: true,
+                    valueFormat: function (d) {
+                        return d3.format(',.1f')(d);
+                    },
+                    duration: 500,
+                    xAxis: {
+                        axisLabel: 'Percentage'
+                    },
+                    yAxis: {
+                        axisLabel: 'Latency (ms)',
+                        axisLabelDistance: -10
+                    }
+                }
+            };
+
+            ctrl.$onChanges = function (data) {
+                var groupedMetrics = {}
+
+                var createPercentageMetrics = function (metricName, window) {
+                    groupedMetrics.percentiles = groupedMetrics['percentiles'] || {}
+                    groupedMetrics.percentiles[metricName] = groupedMetrics.percentiles[metricName] || {};
+                    groupedMetrics.percentiles[metricName][window] = groupedMetrics.percentiles[metricName][window] || {
+                            graphdata: [{
+                                key: window,
+                                values: []
+                            }], numOps: 0
+                        };
+
+                };
+                var metrics = ctrl.jmxdata;
+                for (var key in metrics) {
+                    var percentile = key.match(/(.*Time)(\d+s)(\d+th)PercentileLatency/);
+                    var percentileNumOps = key.match(/(.*Time)(\d+s)NumOps/);
+                    var successFailures = key.match(/(.*)(Success|Failures)/);
+                    var numAverages = key.match(/(.*Time)(NumOps|AvgTime)/);
+                    if (percentile) {
+                        var metricName = percentile[1];
+                        var window = percentile[2];
+                        var percentage = percentile[3]
+                        createPercentageMetrics(metricName, window);
+
+
+                        groupedMetrics.percentiles[metricName][window].graphdata[0]
+                            .values.push({
+                            label: percentage,
+                            value: metrics[key]
+                        })
+                    } else if (successFailures) {
+                        var metricName = successFailures[1];
+                        groupedMetrics.successfailures = groupedMetrics['successfailures'] || {}
+                        groupedMetrics.successfailures[metricName] = groupedMetrics.successfailures[metricName] || {
+                                success: 0,
+                                failures: 0
+                            };
+                        if (successFailures[2] == 'Success') {
+                            groupedMetrics.successfailures[metricName].success = metrics[key];
+                        } else {
+                            groupedMetrics.successfailures[metricName].failures = metrics[key];
+                        }
+
+                    } else if (numAverages) {
+                        var metricName = numAverages[1];
+                        groupedMetrics.numavgs = groupedMetrics['numavgs'] || {}
+                        groupedMetrics.numavgs[metricName] = groupedMetrics.numavgs[metricName] || {
+                                numOps: 0,
+                                avgTime: 0
+                            };
+                        if (numAverages[2] == 'NumOps') {
+                            groupedMetrics.numavgs[metricName].numOps = metrics[key];
+                        } else {
+                            groupedMetrics.numavgs[metricName].avgTime = metrics[key];
+                        }
+
+                    } else if (percentileNumOps) {
+                        var metricName = percentileNumOps[1];
+                        var window = percentileNumOps[2];
+                        createPercentageMetrics(metricName, window);
+                        groupedMetrics.percentiles[metricName][window].numOps = metrics[key];
+                    } else if (isIgnoredJmxKeys(key)) {
+                        //ignore
+                    } else {
+                        groupedMetrics.others = groupedMetrics.others || [];
+                        groupedMetrics.others.push({
+                            'key': key,
+                            'value': metrics[key]
+                        });
+                    }
+
+                }
+                ctrl.metrics = groupedMetrics;
+            };
+
+        }
+    });
+    angular.module('ozone')
+        .component('tabs', {
+            transclude: true,
+            controller: function ($scope) {
+                var ctrl = this;
+                var panes = this.panes = [];
+                this.select = function (pane) {
+                    angular.forEach(panes, function (pane) {
+                        pane.selected = false;
+                    });
+                    pane.selected = true;
+                };
+                this.addPane = function (pane) {
+                    if (panes.length === 0) {
+                        this.select(pane);
+                    }
+                    panes.push(pane);
+                };
+                this.click = function(pane) {
+                    ctrl.select(pane);
+                }
+            },
+            template: '<div class="nav navtabs"><div class="container"><ul class="nav nav-pills">' +
+            '<li ng-repeat="pane in $ctrl.panes" ng-class="{active:pane.selected}">' +
+            '<a href="" ng-click="$ctrl.click(pane)">{{pane.title}}</a> ' +
+            '</li> </ul></div><br/><div class="tab-content" ng-transclude></div> </div>'
+        })
+        .component('pane', {
+            transclude: true,
+            require: {
+                tabsCtrl: '^tabs'
+            },
+            bindings: {
+                title: '@'
+            },
+            controller: function () {
+                this.$onInit = function () {
+                    this.tabsCtrl.addPane(this);
+                };
+            },
+            template: '<div class="tab-pane" ng-if="$ctrl.selected" ng-transclude></div>'
+        });
+
+    angular.module('ozone').component('commonTools', {
+        templateUrl: '/static/templates/tools.html'
+    });
+})();

+ 0 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/ksm/jvm.html → hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/templates/jvm.html


+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/ksm/overview.html → hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/templates/overview.html

@@ -35,3 +35,5 @@
 <h2>JVM parameters</h2>
 
 <jvm-parameters></jvm-parameters>
+
+<div ng-transclude></div>

+ 1 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/ksm/rpc-metrics.html → hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/templates/rpc-metrics.html

@@ -14,10 +14,7 @@
    See the License for the specific language governing permissions and
    limitations under the License.
 -->
-
-<h1>RPC Metrics (port: {{$ctrl.jmxdata['tag.port']}})</h1>
-
-<div ng-hide="$ctrl.metrics.percentiles.length" class="alert alert-info">
+<div ng-hide="$ctrl.metrics.percentiles" class="alert alert-info">
     Please set <b>rpc.metrics.quantile.enable</b> to <b>true</b> and define the
     intervals in seconds with setting <b>rpc.metrics.percentiles.intervals</b>
     (eg. set to <b>60,300</b>) in your hdfs-site.xml

+ 38 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/templates/tools.html

@@ -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.
+-->
+<div id="navbar" class="collapse navbar-collapse">
+    <ul class="nav navbar-nav" id="ui-tabs">
+        <li>
+            <a class="dropdown-toggle" id="toolsMenu"
+               data-toggle="dropdown" aria-haspopup="true"
+               aria-expanded="true" href="#">
+                Common tools
+                <span class="caret"></span>
+            </a>
+            <ul class="dropdown-menu" aria-labelledby="toolsMenu">
+                <li><a href="/jmx">JMX <span
+                        aria-hidden="true"></span></a></li>
+                <li><a href="/conf">Config <span
+                        aria-hidden="true"></a></li>
+                <li><a href="/stacks">Stacks <span
+                        aria-hidden="true"></a></li>
+                <li><a href="/logLevel">Log levels <span
+                        aria-hidden="true"></a></li>
+            </ul>
+        </li>
+    </ul>
+</div><!--/.nav-collapse -->

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java

@@ -75,12 +75,14 @@ public class TestSCMMXBean {
   public void testSCMMXBean() throws Exception {
     ObjectName bean = new ObjectName(
         "Hadoop:service=StorageContainerManager,"
-            + "name=StorageContainerManagerInfo");
+            + "name=StorageContainerManagerInfo,"
+            + "component=ServerRuntime");
 
     String dnRpcPort = (String)mbs.getAttribute(bean,
         "DatanodeRpcPort");
     assertEquals(scm.getDatanodeRpcPort(), dnRpcPort);
 
+
     String clientRpcPort = (String)mbs.getAttribute(bean,
         "ClientRpcPort");
     assertEquals(scm.getClientRpcPort(), clientRpcPort);