You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by li...@apache.org on 2019/07/06 15:46:50 UTC

[zeppelin] branch master updated: [ZEPPELIN-3784] Cluster management web page

This is an automated email from the ASF dual-hosted git repository.

liuxun pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/zeppelin.git


The following commit(s) were added to refs/heads/master by this push:
     new f6fe360  [ZEPPELIN-3784] Cluster management web page
f6fe360 is described below

commit f6fe360e327b33347844f7cd62c7c18304dfa1a9
Author: Xun Liu <li...@apache.org>
AuthorDate: Sat Jul 6 00:27:34 2019 +0800

    [ZEPPELIN-3784] Cluster management web page
    
    ### What is this PR for?
    1. Show/Hide cluster Manager web page menu
    Zeppelin navbar checks for cluster mode via the `api/cluster/address` REST interface, and adds cluster to the zeppelin `login menu` if it is in cluster mode.
    
    2. Cluster Manager  node web page
        The webpage obtains all the node information in the cluster metadata through the `/api/cluster/nodes`  REST interface.
        * All zeppelin server node informat
        * List of interpreter processes running in each node
    
    3. Cluster Manager  interpreter web page
        Clicking on the interpreter list in the node management page will open the interpreter information page, displaying all the information of the interpreter.
    
    design doc : https://docs.google.com/document/d/1a8QLSyR3M5AhlG1GIYuDTj6bwazeuVDKCRRBm-Qa3Bw/edit#heading=h.ckkbfp9161hv
    
    ### What type of PR is it?
    [Feature]
    
    ### Todos
    * [ ] [ZEPPELIN-4221 View the zeppelin service & interpreter log in the cluster management page](https://issues.apache.org/jira/browse/ZEPPELIN-4221)
    * [ ] [ZEPPELIN-4222 Restart the zeppelin service in the cluster management page](https://issues.apache.org/jira/browse/ZEPPELIN-4222)
    * [ ] [ZEPPELIN-4223 Restart the zeppelin interpreter in the cluster management page](https://issues.apache.org/jira/browse/ZEPPELIN-4223)
    
    ### What is the Jira issue?
    * https://issues.apache.org/jira/browse/ZEPPELIN-3784
    
    ### How should this be tested?
    * [CI Pass](https://travis-ci.org/liuxunorg/zeppelin/builds/552323214)
    
    ### Screenshots (if appropriate)
    ![cluster-manager-web](https://user-images.githubusercontent.com/3677382/60395207-beb2cd00-9b62-11e9-8609-3804aa52ebc0.gif)
    
    ### Questions:
    * Does the licenses files need update?
    * Is there breaking changes for older versions?
    * Does this needs documentation?
    
    Author: Xun Liu <li...@apache.org>
    
    Closes #3390 from liuxunorg/ZEPPELIN-3784 and squashes the following commits:
    
    8b9593be0 [Xun Liu] Removed test code & Removed register(Date.class)
    29ca1f906 [Xun Liu] While don't get idle node meta, launch interpreter on local.
    0baa836b7 [Xun Liu] Register the LocalDateTime type in the ClusterManager Serializer
    e7bc9017b [Xun Liu] Change Date() to LocalDateTime.now().
    7a295ba2e [Xun Liu] Modify `LAST_HEARTBEAT`  to `LATEST_HEARTBEAT`
    b5cf17839 [Xun Liu] [ZEPPELIN-3784] Cluster server node & interpreter process management web page
---
 conf/shiro.ini.template                            |   1 +
 conf/zeppelin-site.xml.template                    |   6 +
 .../zeppelin/integration/AuthenticationIT.java     |   1 +
 .../integration/InterpreterModeActionsIT.java      |   1 +
 .../zeppelin/integration/PersonalizeActionsIT.java |   1 +
 .../apache/zeppelin/cluster/ClusterManager.java    |  26 ++-
 .../zeppelin/cluster/ClusterManagerServer.java     |  13 +-
 .../apache/zeppelin/cluster/ClusterMonitor.java    |  20 +-
 .../apache/zeppelin/cluster/meta/ClusterMeta.java  |   6 +-
 .../remote/RemoteInterpreterServer.java            |   7 +-
 .../zeppelin/cluster/ClusterSingleNodeTest.java    |   2 +-
 .../launcher/ClusterInterpreterLauncher.java       |   2 +-
 .../org/apache/zeppelin/rest/ClusterRestApi.java   | 186 +++++++++++++++++
 .../apache/zeppelin/cluster/ClusterEventTest.java  |   1 +
 .../apache/zeppelin/rest/AbstractTestRestApi.java  |   2 +
 zeppelin-web/src/app/app.js                        |   8 +
 zeppelin-web/src/app/cluster/cluster.controller.js |  94 +++++++++
 zeppelin-web/src/app/cluster/cluster.css           | 226 +++++++++++++++++++++
 zeppelin-web/src/app/cluster/cluster.html          | 109 ++++++++++
 zeppelin-web/src/app/cluster/node-status.js        |  56 +++++
 zeppelin-web/src/app/cluster/node.controller.js    |  70 +++++++
 zeppelin-web/src/app/cluster/node.css              | 132 ++++++++++++
 zeppelin-web/src/app/cluster/node.html             | 103 ++++++++++
 .../src/components/navbar/navbar.controller.js     |  11 +
 zeppelin-web/src/components/navbar/navbar.html     |   1 +
 zeppelin-web/src/index.js                          |   2 +
 26 files changed, 1064 insertions(+), 23 deletions(-)

diff --git a/conf/shiro.ini.template b/conf/shiro.ini.template
index 23aa473..1196ba4 100644
--- a/conf/shiro.ini.template
+++ b/conf/shiro.ini.template
@@ -110,6 +110,7 @@ admin = *
 # uncomment the line second last line (/** = anon) and comment the last line (/** = authc)
 #
 /api/version = anon
+/api/cluster/address = anon
 # Allow all authenticated users to restart interpreters on a notebook page.
 # Comment out the following line if you would like to authorize only admin users to restart interpreters.
 /api/interpreter/setting/restart/** = authc
diff --git a/conf/zeppelin-site.xml.template b/conf/zeppelin-site.xml.template
index e53790d..6418340 100755
--- a/conf/zeppelin-site.xml.template
+++ b/conf/zeppelin-site.xml.template
@@ -20,6 +20,12 @@
 <configuration>
 
 <property>
+  <name>zeppelin.cluster.addr</name>
+  <value></value>
+  <description>Server cluster address, eg. 127.0.0.1:6000,127.0.0.2:6000,127.0.0.3:6000</description>
+</property>
+
+<property>
   <name>zeppelin.server.addr</name>
   <value>0.0.0.0</value>
   <description>Server address</description>
diff --git a/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/AuthenticationIT.java b/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/AuthenticationIT.java
index ea6ad69..2e34fd4 100644
--- a/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/AuthenticationIT.java
+++ b/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/AuthenticationIT.java
@@ -70,6 +70,7 @@ public class AuthenticationIT extends AbstractZeppelinIT {
       "finance = *\n" +
       "[urls]\n" +
       "/api/version = anon\n" +
+      "/api/cluster/address = anon\n" +
       "/api/interpreter/** = authc, anyofrolesuser[admin, finance]\n" +
       "/** = authc";
 
diff --git a/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/InterpreterModeActionsIT.java b/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/InterpreterModeActionsIT.java
index 1f18a65..8d9c22f 100644
--- a/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/InterpreterModeActionsIT.java
+++ b/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/InterpreterModeActionsIT.java
@@ -66,6 +66,7 @@ public class InterpreterModeActionsIT extends AbstractZeppelinIT {
       "admin = *\n" +
       "[urls]\n" +
       "/api/version = anon\n" +
+      "/api/cluster/address = anon\n" +
       "/** = authc";
 
   static String originalShiro = "";
diff --git a/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/PersonalizeActionsIT.java b/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/PersonalizeActionsIT.java
index 3133564..fce9466 100644
--- a/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/PersonalizeActionsIT.java
+++ b/zeppelin-integration/src/test/java/org/apache/zeppelin/integration/PersonalizeActionsIT.java
@@ -64,6 +64,7 @@ public class PersonalizeActionsIT extends AbstractZeppelinIT {
       "user = *\n" +
       "[urls]\n" +
       "/api/version = anon\n" +
+      "/api/cluster/address = anon\n" +
       "/** = authc";
 
   static String originalShiro = "";
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterManager.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterManager.java
index 2b2cd50..6389b7c 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterManager.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterManager.java
@@ -91,11 +91,12 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.net.InetAddress;
 import java.net.SocketException;
 import java.net.UnknownHostException;
 import java.time.Instant;
 
-import java.util.Date;
+import java.time.LocalDateTime;
 import java.util.Collections;
 import java.util.Map;
 import java.util.HashMap;
@@ -148,6 +149,8 @@ public abstract class ClusterManager {
 
   protected ClusterMonitor clusterMonitor = null;
 
+  protected boolean isTest = false;
+
   public ClusterManager() {
     try {
       zeplServerHost = RemoteInterpreterUtils.findAvailableHostAddress();
@@ -308,7 +311,20 @@ public abstract class ClusterManager {
   }
 
   public String getClusterNodeName() {
-    return zeplServerHost + ":" + raftServerPort;
+    if (isTest) {
+      // Start three cluster servers in the test case at the same time,
+      // need to avoid duplicate names
+      return this.zeplServerHost + ":" + this.raftServerPort;
+    }
+
+    String hostName = "";
+    try {
+      InetAddress addr = InetAddress.getLocalHost();
+      hostName = addr.getHostName().toString();
+    } catch (IOException e) {
+      LOGGER.error(e.getMessage(), e);
+    }
+    return hostName;
   }
 
   // put metadata into cluster metadata
@@ -469,7 +485,7 @@ public abstract class ClusterManager {
       .register(ArrayList.class)
       .register(HashMap.class)
       .register(ClusterMetaEntity.class)
-      .register(Date.class)
+      .register(LocalDateTime.class)
       .register(Collections.emptyList().getClass())
       .register(HashSet.class)
       .register(DefaultRaftMember.class)
@@ -497,7 +513,7 @@ public abstract class ClusterManager {
       .register(ClusterMetaEntity.class)
       .register(HashMap.class)
       .register(HashSet.class)
-      .register(Date.class)
+      .register(LocalDateTime.class)
       .register(DefaultRaftMember.class)
       .register(MemberId.class)
       .register(RaftMember.Type.class)
@@ -513,7 +529,7 @@ public abstract class ClusterManager {
       .register(ClusterMetaOperation.class)
       .register(ClusterMetaType.class)
       .register(HashMap.class)
-      .register(Date.class)
+      .register(LocalDateTime.class)
       .register(Maps.immutableEntry(new String(), new Object()).getClass())
       .build());
 }
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterManagerServer.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterManagerServer.java
index e8434fb..0305ce6 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterManagerServer.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterManagerServer.java
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.time.Duration;
+import java.time.LocalDateTime;
 import java.util.*;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
@@ -64,9 +65,6 @@ public class ClusterManagerServer extends ClusterManager {
 
   protected MessagingService messagingService = null;
 
-  // Connect to the interpreter process that has been created
-  public static String CONNET_EXISTING_PROCESS = "CONNET_EXISTING_PROCESS";
-
   private List<ClusterEventListener> clusterIntpEventListeners = new ArrayList<>();
   private List<ClusterEventListener> clusterNoteEventListeners = new ArrayList<>();
   private List<ClusterEventListener> clusterAuthEventListeners = new ArrayList<>();
@@ -107,6 +105,7 @@ public class ClusterManagerServer extends ClusterManager {
 
   @VisibleForTesting
   public void initTestCluster(String clusterAddrList, String host, int port) {
+    isTest = true;
     this.zeplServerHost = host;
     this.raftServerPort = port;
 
@@ -210,6 +209,14 @@ public class ClusterManagerServer extends ClusterManager {
         messagingService.registerHandler(CLUSTER_AUTH_EVENT_TOPIC,
             subscribeClusterAuthEvent, MoreExecutors.directExecutor());
 
+        HashMap<String, Object> meta = new HashMap<String, Object>();
+        String nodeName = getClusterNodeName();
+        meta.put(ClusterMeta.NODE_NAME, nodeName);
+        meta.put(ClusterMeta.SERVER_HOST, zeplServerHost);
+        meta.put(ClusterMeta.SERVER_PORT, raftServerPort);
+        meta.put(ClusterMeta.SERVER_START_TIME, LocalDateTime.now());
+        putClusterMeta(SERVER_META, nodeName, meta);
+
         LOGGER.info("RaftServer run() <<<");
       }
     }).start();
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterMonitor.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterMonitor.java
index 4fe8d98..cb8fbaf 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterMonitor.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/ClusterMonitor.java
@@ -24,6 +24,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.lang.management.ManagementFactory;
+import java.time.Duration;
+import java.time.LocalDateTime;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -130,7 +132,8 @@ public class ClusterMonitor {
     if (LOGGER.isDebugEnabled()) {
       LOGGER.debug("checkHealthy()");
     }
-    Date now = new Date();
+
+    LocalDateTime now = LocalDateTime.now();
     // check machine mate
     for (ClusterMetaType metaType : ClusterMetaType.values()) {
       Map<String, HashMap<String, Object>> clusterMeta
@@ -150,11 +153,12 @@ public class ClusterMonitor {
           continue;
         }
 
-        Object heartbeat = meta.get(ClusterMeta.HEARTBEAT);
-        if (heartbeat instanceof Date) {
-          Date dHeartbeat = (Date) heartbeat;
-          long diff = now.getTime() - dHeartbeat.getTime();
-          if (diff > heartbeatTimeout) {
+        Object heartbeat = meta.get(ClusterMeta.LATEST_HEARTBEAT);
+        if (heartbeat instanceof LocalDateTime) {
+          LocalDateTime dHeartbeat = (LocalDateTime) heartbeat;
+          Duration duration = Duration.between(dHeartbeat, now);
+          long timeInterval = duration.getSeconds();
+          if (timeInterval > heartbeatTimeout) {
             // Set the metadata for the heartbeat timeout to offline
             // Cannot delete metadata
             HashMap<String, Object> mapValues = new HashMap<>();
@@ -173,7 +177,7 @@ public class ClusterMonitor {
   // indicating that the process is still active.
   private void sendHeartbeat() {
     HashMap<String, Object> mapMonitorUtil = new HashMap<>();
-    mapMonitorUtil.put(ClusterMeta.HEARTBEAT, new Date());
+    mapMonitorUtil.put(ClusterMeta.LATEST_HEARTBEAT, LocalDateTime.now());
     mapMonitorUtil.put(ClusterMeta.STATUS, ClusterMeta.ONLINE_STATUS);
 
     clusterManager.putClusterMeta(INTP_PROCESS_META, metaKey, mapMonitorUtil);
@@ -212,7 +216,7 @@ public class ClusterMonitor {
     mapMonitorUtil.put(ClusterMeta.MEMORY_CAPACITY, avgMonitorUtil.memoryCapacity);
     mapMonitorUtil.put(ClusterMeta.CPU_USED, avgMonitorUtil.cpuUsed);
     mapMonitorUtil.put(ClusterMeta.CPU_CAPACITY, avgMonitorUtil.cpuCapacity);
-    mapMonitorUtil.put(ClusterMeta.HEARTBEAT, new Date());
+    mapMonitorUtil.put(ClusterMeta.LATEST_HEARTBEAT, LocalDateTime.now());
     mapMonitorUtil.put(ClusterMeta.STATUS, ClusterMeta.ONLINE_STATUS);
 
     String clusterName = clusterManager.getClusterNodeName();
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/meta/ClusterMeta.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/meta/ClusterMeta.java
index e862635..0dfb2b7 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/meta/ClusterMeta.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/cluster/meta/ClusterMeta.java
@@ -40,7 +40,7 @@ public class ClusterMeta implements Serializable {
   public static String SERVER_START_TIME    = "SERVER_START_TIME";
 
   // interperter-process meta
-  public static String INTP_PROCESS_ID      = "INTP_PROCESS_ID";
+  public static String INTP_PROCESS_NAME    = "INTP_PROCESS_NAME";
   public static String INTP_TSERVER_HOST    = "INTP_TSERVER_HOST";
   public static String INTP_TSERVER_PORT    = "INTP_TSERVER_PORT";
   public static String INTP_START_TIME      = "INTP_START_TIME";
@@ -51,12 +51,14 @@ public class ClusterMeta implements Serializable {
   public static String MEMORY_CAPACITY      = "MEMORY_CAPACITY";
   public static String MEMORY_USED          = "MEMORY_USED";
 
-  public static String HEARTBEAT            = "HEARTBEAT";
+  public static String LATEST_HEARTBEAT     = "LATEST_HEARTBEAT";
 
   // zeppelin-server or interperter-process status
   public static String STATUS               = "STATUS";
   public static String ONLINE_STATUS        = "ONLINE";
   public static String OFFLINE_STATUS       = "OFFLINE";
+  public static String INTP_PROCESS_COUNT   = "INTP_PROCESS_COUNT";
+  public static String INTP_PROCESS_LIST    = "INTP_PROCESS_LIST";
 
   // cluster_name = host:port
   // Map:cluster_name -> {server_tserver_host,server_tserver_port,cpu_capacity,...}
diff --git a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterServer.java b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterServer.java
index 3dc41f4..d7e946b 100644
--- a/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterServer.java
+++ b/zeppelin-interpreter/src/main/java/org/apache/zeppelin/interpreter/remote/RemoteInterpreterServer.java
@@ -87,6 +87,7 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.URL;
 import java.nio.ByteBuffer;
+import java.time.LocalDateTime;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Date;
@@ -332,11 +333,11 @@ public class RemoteInterpreterServer extends Thread
     // commit interpreter meta
     HashMap<String, Object> meta = new HashMap<>();
     meta.put(ClusterMeta.NODE_NAME, nodeName);
-    meta.put(ClusterMeta.INTP_PROCESS_ID, interpreterGroupId);
+    meta.put(ClusterMeta.INTP_PROCESS_NAME, interpreterGroupId);
     meta.put(ClusterMeta.INTP_TSERVER_HOST, host);
     meta.put(ClusterMeta.INTP_TSERVER_PORT, port);
-    meta.put(ClusterMeta.INTP_START_TIME, new Date());
-    meta.put(ClusterMeta.HEARTBEAT, new Date());
+    meta.put(ClusterMeta.INTP_START_TIME, LocalDateTime.now());
+    meta.put(ClusterMeta.LATEST_HEARTBEAT, LocalDateTime.now());
     meta.put(ClusterMeta.STATUS, ClusterMeta.ONLINE_STATUS);
 
     clusterManagerClient.putClusterMeta(ClusterMetaType.INTP_PROCESS_META, interpreterGroupId, meta);
diff --git a/zeppelin-interpreter/src/test/java/org/apache/zeppelin/cluster/ClusterSingleNodeTest.java b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/cluster/ClusterSingleNodeTest.java
index 59853d4..d96f6ba 100644
--- a/zeppelin-interpreter/src/test/java/org/apache/zeppelin/cluster/ClusterSingleNodeTest.java
+++ b/zeppelin-interpreter/src/test/java/org/apache/zeppelin/cluster/ClusterSingleNodeTest.java
@@ -109,7 +109,7 @@ public class ClusterSingleNodeTest {
     HashMap hashMap = (HashMap) meta;
 
     // Get metadata for the current service
-    Object values = hashMap.get(zServerHost + ":" + zServerPort);
+    Object values = hashMap.get(clusterClient.getClusterNodeName());
     assertEquals(true, (values instanceof HashMap));
     HashMap mapMetaValues = (HashMap) values;
 
diff --git a/zeppelin-plugins/launcher/cluster/src/main/java/org/apache/zeppelin/interpreter/launcher/ClusterInterpreterLauncher.java b/zeppelin-plugins/launcher/cluster/src/main/java/org/apache/zeppelin/interpreter/launcher/ClusterInterpreterLauncher.java
index 83f4232..2b2ac61 100644
--- a/zeppelin-plugins/launcher/cluster/src/main/java/org/apache/zeppelin/interpreter/launcher/ClusterInterpreterLauncher.java
+++ b/zeppelin-plugins/launcher/cluster/src/main/java/org/apache/zeppelin/interpreter/launcher/ClusterInterpreterLauncher.java
@@ -89,7 +89,7 @@ public class ClusterInterpreterLauncher extends StandardInterpreterLauncher
       HashMap<String, Object> meta = clusterServer.getIdleNodeMeta();
       if (null == meta) {
         LOGGER.error("Don't get idle node meta, launch interpreter on local.");
-        super.launch(context);
+        return createInterpreterProcess(context);
       }
 
       String srvHost = (String) meta.get(SERVER_HOST);
diff --git a/zeppelin-server/src/main/java/org/apache/zeppelin/rest/ClusterRestApi.java b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/ClusterRestApi.java
new file mode 100644
index 0000000..ef81a13
--- /dev/null
+++ b/zeppelin-server/src/main/java/org/apache/zeppelin/rest/ClusterRestApi.java
@@ -0,0 +1,186 @@
+/*
+ * 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.zeppelin.rest;
+
+import com.google.gson.Gson;
+import org.apache.zeppelin.annotation.ZeppelinApi;
+import org.apache.zeppelin.cluster.ClusterManagerServer;
+import org.apache.zeppelin.cluster.meta.ClusterMeta;
+import org.apache.zeppelin.cluster.meta.ClusterMetaType;
+import org.apache.zeppelin.conf.ZeppelinConfiguration;
+import org.apache.zeppelin.server.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Response;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * clusters Rest api.
+ */
+@Path("/cluster")
+@Produces("application/json")
+public class ClusterRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(ClusterRestApi.class);
+  Gson gson = new Gson();
+
+  private ClusterManagerServer clusterManagerServer = ClusterManagerServer.getInstance();
+
+  // Do not modify, Use by `zeppelin-web/src/app/cluster/cluster.html`
+  private static String PROPERTIES = "properties";
+
+  @GET
+  @Path("/address")
+  @ZeppelinApi
+  public Response getClusterAddress() {
+    ZeppelinConfiguration zconf = ZeppelinConfiguration.create();
+    String clusterAddr = zconf.getClusterAddress();
+    Map<String, String> data = new HashMap<>();
+    data.put("clusterAddr", clusterAddr);
+
+    return new JsonResponse<>(Response.Status.OK, "Cluster Address", data).build();
+  }
+
+  /**
+   * get all nodes of clusters
+   */
+  @GET
+  @Path("/nodes")
+  @ZeppelinApi
+  public Response getClusterNodes(){
+    ArrayList<HashMap<String, Object>> nodes = new ArrayList<HashMap<String, Object>>();
+
+    Map<String, HashMap<String, Object>> clusterMeta = null;
+    Map<String, HashMap<String, Object>> intpMeta = null;
+    clusterMeta = clusterManagerServer.getClusterMeta(ClusterMetaType.SERVER_META, "");
+    intpMeta = clusterManagerServer.getClusterMeta(ClusterMetaType.INTP_PROCESS_META, "");
+
+    // Number of calculation processes
+    for (Map.Entry<String, HashMap<String, Object>> serverMetaEntity : clusterMeta.entrySet()) {
+      if (!serverMetaEntity.getValue().containsKey(ClusterMeta.NODE_NAME)) {
+        continue;
+      }
+      String serverNodeName = (String) serverMetaEntity.getValue().get(ClusterMeta.NODE_NAME);
+
+      ArrayList<String> arrIntpProcess = new ArrayList<>();
+      int intpProcCount = 0;
+      for (Map.Entry<String, HashMap<String, Object>> intpMetaEntity : intpMeta.entrySet()) {
+        if (!intpMetaEntity.getValue().containsKey(ClusterMeta.NODE_NAME)
+            && !intpMetaEntity.getValue().containsKey(ClusterMeta.INTP_PROCESS_NAME)) {
+          continue;
+        }
+        String intpNodeName = (String) intpMetaEntity.getValue().get(ClusterMeta.NODE_NAME);
+
+        if (serverNodeName.equals(intpNodeName)) {
+          intpProcCount ++;
+          String intpName = (String) intpMetaEntity.getValue().get(ClusterMeta.INTP_PROCESS_NAME);
+          arrIntpProcess.add(intpName);
+        }
+      }
+      serverMetaEntity.getValue().put(ClusterMeta.INTP_PROCESS_COUNT, intpProcCount);
+      serverMetaEntity.getValue().put(ClusterMeta.INTP_PROCESS_LIST, arrIntpProcess);
+    }
+
+    for (Map.Entry<String, HashMap<String, Object>> entry : clusterMeta.entrySet()) {
+      String nodeName = entry.getKey();
+      Map<String, Object> properties = entry.getValue();
+
+      Map<String, Object> sortProperties = new HashMap<>();
+
+      if (properties.containsKey(ClusterMeta.CPU_USED)
+          && properties.containsKey(ClusterMeta.CPU_CAPACITY)) {
+        float cpuUsed = (long) properties.get(ClusterMeta.CPU_USED) / (float) 100.0;
+        float cpuCapacity = (long) properties.get(ClusterMeta.CPU_CAPACITY) / (float) 100.0;
+        float cpuRate = cpuUsed / cpuCapacity * 100;
+
+        String cpuInfo = String.format("%.2f / %.2f = %.2f", cpuUsed, cpuCapacity, cpuRate);
+        sortProperties.put(ClusterMeta.CPU_USED + " / " + ClusterMeta.CPU_CAPACITY, cpuInfo + "%");
+      }
+
+      if (properties.containsKey(ClusterMeta.MEMORY_USED)
+          && properties.containsKey(ClusterMeta.MEMORY_CAPACITY)) {
+        float memoryUsed = (long) properties.get(ClusterMeta.MEMORY_USED) / (float) (1024*1024*1024);
+        float memoryCapacity = (long) properties.get(ClusterMeta.MEMORY_CAPACITY) / (float) (1024*1024*1024);
+        float memoryRate = memoryUsed / memoryCapacity * 100;
+
+        String memoryInfo = String.format("%.2fGB / %.2fGB = %.2f",
+            memoryUsed, memoryCapacity, memoryRate);
+        sortProperties.put(ClusterMeta.MEMORY_USED + " / " + ClusterMeta.MEMORY_CAPACITY, memoryInfo + "%");
+      }
+
+      if (properties.containsKey(ClusterMeta.SERVER_START_TIME)) {
+        sortProperties.put(ClusterMeta.SERVER_START_TIME, properties.get(ClusterMeta.SERVER_START_TIME));
+      }
+      if (properties.containsKey(ClusterMeta.STATUS)) {
+        sortProperties.put(ClusterMeta.STATUS, properties.get(ClusterMeta.STATUS));
+      }
+      if (properties.containsKey(ClusterMeta.LATEST_HEARTBEAT)) {
+        sortProperties.put(ClusterMeta.LATEST_HEARTBEAT, properties.get(ClusterMeta.LATEST_HEARTBEAT));
+      }
+      if (properties.containsKey(ClusterMeta.INTP_PROCESS_LIST)) {
+        sortProperties.put(ClusterMeta.INTP_PROCESS_LIST, properties.get(ClusterMeta.INTP_PROCESS_LIST));
+      }
+
+      HashMap<String, Object> node = new HashMap<String, Object>();
+      node.put(ClusterMeta.NODE_NAME, nodeName);
+      node.put(PROPERTIES, sortProperties);
+
+      nodes.add(node);
+    }
+
+    return new JsonResponse(Response.Status.OK, "", nodes).build();
+  }
+
+  private String formatIntpLink(String intpName) {
+    return String.format("<a href=\"/#/cluster/%s\">%s</a>", intpName, intpName);
+  }
+
+  /**
+   * get node info by id
+   */
+  @GET
+  @Path("/node/{nodeName}/{intpName}")
+  @ZeppelinApi
+  public Response getClusterNode(@PathParam("nodeName") String nodeName,
+                                 @PathParam("intpName") String intpName){
+    ArrayList<HashMap<String, Object>> intpProcesses = new ArrayList<HashMap<String, Object>>();
+
+    Map<String, HashMap<String, Object>> intpMeta = null;
+    intpMeta = clusterManagerServer.getClusterMeta(ClusterMetaType.INTP_PROCESS_META, "");
+
+    // Number of calculation processes
+    for (Map.Entry<String, HashMap<String, Object>> intpMetaEntity : intpMeta.entrySet()) {
+      String intpNodeName = (String) intpMetaEntity.getValue().get(ClusterMeta.NODE_NAME);
+
+      if (null != intpNodeName && intpNodeName.equals(nodeName)) {
+        HashMap<String, Object> node = new HashMap<String, Object>();
+        node.put(ClusterMeta.NODE_NAME, intpNodeName);
+        node.put(PROPERTIES, intpMetaEntity.getValue());
+        intpProcesses.add(node);
+      }
+    }
+
+    return new JsonResponse(Response.Status.OK, "", intpProcesses).build();
+  }
+}
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/cluster/ClusterEventTest.java b/zeppelin-server/src/test/java/org/apache/zeppelin/cluster/ClusterEventTest.java
index 7e18987..169460a 100644
--- a/zeppelin-server/src/test/java/org/apache/zeppelin/cluster/ClusterEventTest.java
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/cluster/ClusterEventTest.java
@@ -302,6 +302,7 @@ public class ClusterEventTest extends ZeppelinServerMock {
       }
     }
   }
+
   @Test
   public void testCloneNoteEvent() throws IOException {
     Note note1 = null;
diff --git a/zeppelin-server/src/test/java/org/apache/zeppelin/rest/AbstractTestRestApi.java b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/AbstractTestRestApi.java
index a36b419..41d5970 100644
--- a/zeppelin-server/src/test/java/org/apache/zeppelin/rest/AbstractTestRestApi.java
+++ b/zeppelin-server/src/test/java/org/apache/zeppelin/rest/AbstractTestRestApi.java
@@ -85,6 +85,7 @@ public abstract class AbstractTestRestApi {
           "admin = *\n" +
           "[urls]\n" +
           "/api/version = anon\n" +
+          "/api/cluster/address = anon\n" +
           "/** = authc";
 
   private static String zeppelinShiroKnox =
@@ -108,6 +109,7 @@ public abstract class AbstractTestRestApi {
           "admin = *\n" +
           "[urls]\n" +
           "/api/version = anon\n" +
+          "/api/cluster/address = anon\n" +
           "/** = authc";
 
   private static File knoxSsoPem = null;
diff --git a/zeppelin-web/src/app/app.js b/zeppelin-web/src/app/app.js
index 826829a..53d6645 100644
--- a/zeppelin-web/src/app/app.js
+++ b/zeppelin-web/src/app/app.js
@@ -136,6 +136,14 @@ let zeppelinWebApp = angular.module('zeppelinWebApp', requiredModules)
         templateUrl: 'app/search/result-list.html',
         controller: 'SearchResultCtrl',
       })
+      .when('/cluster', {
+        templateUrl: 'app/cluster/cluster.html',
+        controller: 'ClusterCtrl',
+      })
+      .when('/cluster/:nodeName/:intpName', {
+        templateUrl: 'app/cluster/node.html',
+        controller: 'NodeCtrl',
+      })
       .otherwise({
         redirectTo: '/',
       });
diff --git a/zeppelin-web/src/app/cluster/cluster.controller.js b/zeppelin-web/src/app/cluster/cluster.controller.js
new file mode 100644
index 0000000..ce60d81
--- /dev/null
+++ b/zeppelin-web/src/app/cluster/cluster.controller.js
@@ -0,0 +1,94 @@
+/*
+ * Licensed 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 {getNodeColorByStatus, getNodeIconByStatus} from './node-status';
+
+angular.module('zeppelinWebApp').controller('ClusterCtrl', ClusterController);
+
+const NodeDateSorter = {
+  Domain_ASC: 'Domain ASC',
+  Domain_DSC: 'Domain DSC',
+};
+
+function ClusterController($scope, $rootScope, $http, baseUrlSrv, ngToast, websocketMsgSrv) {
+  'ngInject';
+
+  $scope.isFilterLoaded = false;
+  $scope.nodes = [];
+  $scope.sorter = {
+    availableDateSorter: Object.keys(NodeDateSorter).map((key) => {
+      return NodeDateSorter[key];
+    }),
+    currentDateSorter: NodeDateSorter.Domain_ASC,
+  };
+  $scope.filteredNodes = $scope.nodes;
+  $scope.filterConfig = {
+    isRunningAlwaysTop: true,
+    nodeNameFilterValue: '',
+    interpreterFilterValue: '*',
+    isSortByAsc: true,
+  };
+
+  $scope.pagination = {
+    currentPage: 1,
+    itemsPerPage: 10,
+    maxPageCount: 5,
+  };
+
+  ngToast.dismiss();
+  init();
+
+  /** functions */
+
+  $scope.setNodeDateSorter = function(dateSorter) {
+    $scope.sorter.currentDateSorter = dateSorter;
+  };
+
+  $scope.getNodesInCurrentPage = function(nodes) {
+    const cp = $scope.pagination.currentPage;
+    const itp = $scope.pagination.itemsPerPage;
+    return nodes.slice((cp - 1) * itp, (cp * itp));
+  };
+
+  $scope.getNodeIconByStatus = getNodeIconByStatus;
+
+  $scope.getNodeColorByStatus = getNodeColorByStatus;
+
+  $scope.filterNodes = function(nodes, filterConfig) {
+    return nodes;
+  };
+
+  function init() {
+    $http.get(baseUrlSrv.getRestApiBase() + '/cluster/nodes')
+      .success(function(data, status, headers, config) {
+        $scope.nodes = data.body;
+        $scope.filteredNodes = $scope.nodes;
+        console.log(JSON.stringify($scope.nodes));
+      })
+      .error(function(data, status, headers, config) {
+        if (status === 401) {
+          ngToast.danger({
+            content: 'You don\'t have permission on this page',
+            verticalPosition: 'bottom',
+            timeout: '3000',
+          });
+          setTimeout(function() {
+            window.location = baseUrlSrv.getBase();
+          }, 3000);
+        }
+        console.log('Error %o %o', status, data.message);
+      });
+    $scope.isFilterLoaded = true;
+  }
+}
diff --git a/zeppelin-web/src/app/cluster/cluster.css b/zeppelin-web/src/app/cluster/cluster.css
new file mode 100644
index 0000000..498f687
--- /dev/null
+++ b/zeppelin-web/src/app/cluster/cluster.css
@@ -0,0 +1,226 @@
+/*
+ * Licensed 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.
+ */
+
+.cluster-col {
+  margin: 0;
+  padding: 0;
+}
+
+.cluster {
+  padding: 2px 8px 4px 8px;
+  min-height: 32px;
+}
+
+.cluster-manager-header {
+  margin: -10px -10px 20px;
+  padding: 10px 15px 15px 15px;
+  background: white;
+  box-shadow: 0 2px 4px rgba(0, 0, 0, 0.15);
+  border-bottom: 1px solid #E5E5E5;
+}
+
+.cluster-manager-header .header {
+  font-family: 'Roboto', sans-serif;
+}
+
+.cluster-search-tool {
+  display: inline-block;
+  margin-left: 15px;
+}
+
+#cluster-manager-header .cluster-search-tool .search-input {
+  margin-right: 7px;
+  min-width: 215px;
+}
+
+#cluster-manager-header .cluster-search-tool .search-input > input {
+  font-family: 'FontAwesome', 'Helvetica Neue', Helvetica, Arial, sans-serif;
+  border-radius: 4px;
+  font-size: 14px;
+  padding-left: 10px;
+}
+
+#cluster-manager-header .cluster-search-tool .dropdown-toggle {
+  border-radius: 3px;
+  float: none;
+  min-width: 150px;
+  text-align: left;
+  margin-right: 5px;
+}
+
+
+#cluster-manager-header .cluster-search-tool .date-sort-button {
+  min-width: 180px;
+}
+
+.cluster-search-tool .dropdown-text-desc {
+  color: gray;
+  font-weight: 400;
+  font-family: "Roboto", "Helvetica Neue", Helvetica, Arial, sans-serif;
+}
+
+.cluster-search-tool .dropdown-text-value {
+  margin-left: 2px;
+  font-family: "Roboto", "Helvetica Neue", Helvetica, Arial, sans-serif;
+}
+
+.search-tool-dropdown-content > li > a {
+  border-bottom: 1px solid #eee;
+}
+
+.search-tool-dropdown-content > li:last-child > a {
+  border-bottom: none;
+}
+
+.cluster-icon-desc-container {
+  display: inline-block;
+  margin-top: 10px;
+  margin-right: 30px;
+  float: right;
+}
+
+.cluster-desc-icon {
+  margin-right: 3px;
+}
+
+.cluster-pagination-container {
+  text-align: center;
+  margin-top: 50px;
+  padding-bottom: 100px;
+}
+
+.cluster-counter {
+  float: right;
+  clear: both;
+  margin-left: 8px;
+  margin-top: 6px;
+}
+
+.cluster-counter .cluster-counter-label {
+  color: gray;
+}
+
+.cluster-counter .cluster-counter-value {
+  font-size: 15px;
+  font-weight: bold;
+}
+
+/*
+  node Style
+*/
+
+.node-space {
+  margin-bottom: 5px !important;
+  padding: 10px 10px 10px 10px !important;
+  min-height: 30px;
+}
+
+.node-margin {
+  margin-right: 2px;
+  margin-left: 2px;
+}
+
+.node-types i {
+  font-weight: bold;
+  font-size: 10px;
+}
+
+
+/*
+  node Controls CSS
+*/
+
+.node .runControl {
+  font-size: 1px;
+  color: #AAAAAA;
+  height:4px;
+  margin: 0px 0px 0px 0px;
+}
+
+.node .runControl .progress {
+  position: relative;
+  width: 100%;
+  height: 4px;
+  z-index: 100;
+  border-radius: 0;
+}
+
+.node .control span {
+  margin-left: 4px;
+}
+
+.node .control {
+  background: rgba(255,255,255,0.85);
+  float: right;
+  color: #999;
+  margin-top: 1px;
+  margin-right: 5px;
+  position: absolute;
+  clear: both;
+  right: 15px;
+  text-align: right;
+  font-size: 12px;
+  padding: 4px;
+}
+
+.node .control li {
+  font-size: 12px;
+  margin-bottom: 4px;
+  color: #333333;
+}
+
+.node .control .tooltip {
+  z-index: 10003;
+}
+
+.node .control .node-control-btn {
+  cursor: pointer;
+  color: #3071A9;
+}
+
+@-webkit-keyframes spinnerRotateAnimation
+{
+  from{-webkit-transform:rotate(0deg);}
+  to{-webkit-transform:rotate(360deg);}
+}
+@-moz-keyframes spinnerRotateAnimation
+{
+  from{-moz-transform:rotate(0deg);}
+  to{-moz-transform:rotate(360deg);}
+}
+@-ms-keyframes spinnerRotateAnimation
+{
+  from{-ms-transform:rotate(0deg);}
+  to{-ms-transform:rotate(360deg);}
+}
+
+@keyframes spinnerRotateAnimation {
+  from {transform: rotate(0deg);}
+  to{transform: rotate(360deg);}
+}
+
+.spinAnimation{
+  -webkit-animation-name: spinnerRotateAnimation;
+  -webkit-animation-duration: 1s;
+  -webkit-animation-iteration-count: infinite;
+  -webkit-animation-timing-function: linear;
+  -moz-animation-name: spinnerRotateAnimation;
+  -moz-animation-duration: 1s;
+  -moz-animation-iteration-count: infinite;
+  -moz-animation-timing-function: linear;
+  -ms-animation-name: spinnerRotateAnimation;
+  -ms-animation-duration: 1s;
+  -ms-animation-iteration-count: infinite;
+  -ms-animation-timing-function: linear;
+}
diff --git a/zeppelin-web/src/app/cluster/cluster.html b/zeppelin-web/src/app/cluster/cluster.html
new file mode 100644
index 0000000..a895f6b
--- /dev/null
+++ b/zeppelin-web/src/app/cluster/cluster.html
@@ -0,0 +1,109 @@
+<!--
+Licensed 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="cluster-manager-header" class="cluster-manager-header">
+  <div class="header">
+    <div class="row">
+      <div class="col-md-12">
+        <h3 class="new_h3">
+          Cluster
+        </h3>
+      </div>
+    </div>
+    <div class="row">
+      <div class="col-md-12">
+        You can monitor the status of cluster and navigate to nodes.
+      </div>
+    </div>
+  </div>
+</div>
+
+<div>
+  <div class="note-jump"></div>
+  <div ng-if="!isFilterLoaded" class="paragraph-col">
+    <div class="cluster-space box cluster-margin text-center">
+      <i style="color: blue" class="fa fa-spinner spinAnimation"></i>
+      Loading...
+    </div>
+  </div>
+  <div ng-if="filteredNodes.length > 0"
+       ng-repeat="node in getNodesInCurrentPage(filteredNodes)"
+       class="paragraph-col">
+    <div class="cluster-space box cluster-margin">
+
+      <div>
+        <a style="text-decoration: none !important;" ng-href="#/cluster/{{node.NODE_NAME}}/all">
+          <h3 class="interpreter-title">{{node.NODE_NAME}}
+            <small ng-switch="node.properties.STATUS">
+              <small ng-switch-when="ONLINE">
+                <i style="color: green; margin-right: 3px;" class="fa fa-circle"
+                   uib-tooltip="online">
+                </i>
+              </small>
+              <small ng-switch-when="OFFLINE">
+                <i style="color: red; cursor: pointer" class="fa fa-circle"
+                   uib-tooltip="offline">
+                </i>
+              </small>
+            </small>
+          </h3>
+        </a>
+      </div>
+
+      <div class="row properties" style="margin-top:20px !important">
+        <div ng-show="_.isEmpty(node.properties) && !valueform.$visible" class="col-md-12 gray40-message">
+          <em>Currently there are no properties this node</em>
+        </div>
+        <div class="col-md-12" ng-show="!_.isEmpty(node.properties) || valueform.$visible">
+          <h5>Properties</h5>
+          <table class="table table-striped" style="margin-bottom: 0px;">
+            <thead>
+            <tr>
+              <th style="width:20%">name</th>
+              <th style="width:80%">value</th>
+            </tr>
+            </thead>
+            <tr ng-repeat="key in node.properties | sortByKey" >
+              <td style="vertical-align: middle;">{{key}}</td>
+              <td style="vertical-align: middle;">
+                <span ng-if="key === 'INTP_PROCESS_LIST'" ng-repeat="value in node.properties[key]">
+                  <a ng-href="#cluster/{{node.NODE_NAME}}/{{value}}">{{value}}</a>,
+                </span>
+                <span ng-if="key !== 'INTP_PROCESS_LIST'">
+                  {{node.properties[key]}}
+                </span>
+              </td>
+            </tr>
+          </table>
+        </div>
+      </div>
+    </div>
+  </div>
+  <div ng-if="isFilterLoaded === false && filteredNodes.length <= 0"
+       class="paragraph-col">
+    <div class="cluster-space box cluster-margin text-center">No cluster found</div>
+  </div>
+
+  <!-- pagination -->
+  <div class="cluster-pagination-container">
+    <ul uib-pagination class="pagination-sm"
+        total-items="filteredNodes.length"
+        ng-model="pagination.currentPage"
+        items-per-page="pagination.itemsPerPage"
+        boundary-links="true" rotate="false"
+        max-size="pagination.maxPageCount"
+        previous-text="&lsaquo;" next-text="&rsaquo;"
+        first-text="&laquo;" last-text="&raquo;"></ul>
+  </div>
+
+</div>
diff --git a/zeppelin-web/src/app/cluster/node-status.js b/zeppelin-web/src/app/cluster/node-status.js
new file mode 100644
index 0000000..fa2b985
--- /dev/null
+++ b/zeppelin-web/src/app/cluster/node-status.js
@@ -0,0 +1,56 @@
+/*
+ * Licensed 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 const NodeStatus = {
+  READY: 'READY',
+  FINISHED: 'FINISHED',
+  ABORT: 'ABORT',
+  ERROR: 'ERROR',
+  PENDING: 'PENDING',
+  RUNNING: 'RUNNING',
+};
+
+export function getNodeIconByStatus(task) {
+  let i = parseInt(task);
+  if (i % 6 === 0) {
+    return 'fa fa-circle-o';
+  } else if (i % 6 === 1) {
+    return 'fa fa-circle';
+  } else if (i % 6 === 2) {
+    return 'fa fa-circle';
+  } else if (i % 6 === 3) {
+    return 'fa fa-circle';
+  } else if (i % 6 === 4) {
+    return 'fa fa-circle';
+  } else if (i % 6 === 5) {
+    return 'fa fa-spinner';
+  }
+}
+
+export function getNodeColorByStatus(task) {
+  let i = parseInt(task);
+  if (i % 6 === 0) {
+    return 'green';
+  } else if (i % 6 === 1) {
+    return 'green';
+  } else if (i % 6 === 2) {
+    return 'orange';
+  } else if (i % 6 === 3) {
+    return 'red';
+  } else if (i % 6 === 4) {
+    return 'gray';
+  } else if (i % 6 === 5) {
+    return 'blue';
+  }
+}
diff --git a/zeppelin-web/src/app/cluster/node.controller.js b/zeppelin-web/src/app/cluster/node.controller.js
new file mode 100644
index 0000000..338ad86
--- /dev/null
+++ b/zeppelin-web/src/app/cluster/node.controller.js
@@ -0,0 +1,70 @@
+/*
+ * Licensed 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.
+ */
+
+angular.module('zeppelinWebApp').controller('NodeCtrl', NodeCtrl);
+
+function NodeCtrl($scope, $routeParams, $http, baseUrlSrv, ngToast) {
+  'ngInject';
+  $scope.nodeName = $routeParams.nodeName;
+  $scope.intpName = $routeParams.intpName;
+  $scope.intpProcesses = [];
+  $scope.pagination = {
+    currentPage: 1,
+    itemsPerPage: 10,
+    maxPageCount: 5,
+  };
+  if ($scope.intpName !== null && $scope.intpName !== '' && $scope.intpName !== 'all') {
+    $scope.searchNode = $scope.intpName;
+  } else {
+    $scope.searchNode = '';
+  }
+  $scope.filteredProcesses=$scope.intpProcesses;
+  $scope.nodeFilter = function(intpProcess) {
+    return intpProcess.properties.INTP_PROCESS_NAME.indexOf($scope.searchNode) !== -1;
+  };
+  $scope._ = _;
+  ngToast.dismiss();
+
+  $scope.getProgressInCurrentPage = function(pros) {
+    $scope.filteredProcesses = pros;
+    const cp = $scope.pagination.currentPage;
+    const itp = $scope.pagination.itemsPerPage;
+    return pros.slice((cp - 1) * itp, (cp * itp));
+  };
+
+  let init = function() {
+    $http.get(baseUrlSrv.getRestApiBase() + '/cluster/node/' + $scope.nodeName + '/' + $scope.intpName)
+      .success(function(data, status, headers, config) {
+        $scope.intpProcesses = data.body;
+        console.log('scope.intpProcesses.length='+$scope.intpProcesses.length);
+        console.log('scope.intpProcesses='+$scope.intpProcesses);
+        console.log(JSON.stringify($scope.intpProcesses));
+      })
+      .error(function(data, status, headers, config) {
+        if (status === 401) {
+          ngToast.danger({
+            content: 'You don\'t have permission on this page',
+            verticalPosition: 'bottom',
+            timeout: '3000',
+          });
+          setTimeout(function() {
+            window.location = baseUrlSrv.getBase();
+          }, 3000);
+        }
+        console.log('Error %o %o', status, data.message);
+      });
+  };
+
+  init();
+}
diff --git a/zeppelin-web/src/app/cluster/node.css b/zeppelin-web/src/app/cluster/node.css
new file mode 100644
index 0000000..4c9e308
--- /dev/null
+++ b/zeppelin-web/src/app/cluster/node.css
@@ -0,0 +1,132 @@
+/*
+ * Licensed 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.
+ */
+
+.nodeHead {
+  margin: -10px -10px 20px;
+  padding: 10px 15px 15px 15px;
+  background: white;
+  box-shadow: 0 2px 4px rgba(0, 0, 0, 0.15);
+  border-bottom: 1px solid #E5E5E5;
+}
+
+.nodeHead .header {
+  font-family: 'Roboto', sans-serif;
+}
+
+.nodeHead textarea,
+.node textarea {
+  width: 100%;
+  display: block;
+  height: 20px;
+  resize: none;
+  border: 1px solid #CCCCCC;
+  font-size: 12px;
+}
+
+.node input {
+  width: 100%;
+  display: block;
+  height: 23px;
+  border: 1px solid #CCCCCC;
+  border-radius: 3px;
+  vertical-align: middle;
+  font-size: 12px;
+}
+
+.node textarea {
+  min-height: 23px;
+  border-radius: 3px;
+  vertical-align: middle;
+}
+
+.node .node-title {
+  font-size: 20px;
+  font-weight: bold;
+  color: #3071a9;
+  float: left;
+  margin-top: 0;
+}
+
+.node ul {
+  margin: 0;
+  padding: 0;
+}
+
+.node .nodeInfo {
+  list-style-type: none;
+}
+
+.node table {
+  table-layout: fixed;
+  word-break: break-all;
+}
+
+.node table tr .nodePropertyKey {
+  padding : 5px 5px 5px 5px;
+}
+
+.node table tr .nodePropertyValue {
+  padding : 5px 5px 5px 5px;
+  display: block;
+  max-height: 100px;
+  overflow-y: auto;
+}
+
+.permissionsForm {
+  list-style-type: none;
+  background: #EFEFEF;
+  padding: 10px 10px 10px 10px;
+  box-shadow: 0 1px 1px rgba(0, 0, 0, 0.15);
+  border: 1px solid #E5E5E5;
+}
+
+.nodeSettingAdd {
+  margin : 5px 5px 5px 5px;
+  padding : 10px 10px 10px 10px;
+}
+
+.editable-wrap {
+  width : 100%;
+}
+
+.node h5 {
+  font-weight: bold;
+}
+
+.new_h3 {
+  margin-top: 1px;
+  padding-top: 7px;
+  float: left;
+}
+
+.gray40-message {
+  color: #666;
+}
+
+.blackOpc:hover {
+  color: #000;
+  opacity: .5;
+}
+
+.remove-margin-top-bottom {
+  margin-top: 0px !important;
+  margin-bottom: 0px !important;
+}
+
+.node-binding-mode-info-link {
+  color: #3071a9;
+  vertical-align: middle;
+  margin-left: 2px;
+  text-decoration: none !important;
+}
diff --git a/zeppelin-web/src/app/cluster/node.html b/zeppelin-web/src/app/cluster/node.html
new file mode 100644
index 0000000..444865a
--- /dev/null
+++ b/zeppelin-web/src/app/cluster/node.html
@@ -0,0 +1,103 @@
+<!--
+Licensed 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 class="nodeHead">
+  <div class="header">
+    <div class="row">
+      <div class="col-md-12">
+        <h3 class="new_h3">
+          {{nodeName}}
+        </h3>
+      </div>
+    </div>
+    <div class="row">
+      <div class="col-md-12">
+        Interpreter process of {{nodeName}}
+      </div>
+    </div>
+    <div class="row">
+      <div class="col-md-4">
+        <div class="input-group" style="margin-top: 10px">
+          <input type="text" ng-model="searchNode"
+                 class="form-control ng-pristine ng-untouched ng-valid ng-empty"
+                 ng-model-options="{ updateOn: 'default blur', debounce: { 'default': 300, 'blur': 0 } }"
+                 placeholder="Search process"/>
+          <span class="input-group-btn">
+            <button type="submit" class="btn btn-default" ng-disabled="!navbar.connected">
+              <i class="glyphicon glyphicon-search"></i>
+            </button>
+          </span>
+        </div>
+      </div>
+    </div>
+  </div>
+</div>
+<div interpreter-item class="box width-full"
+     ng-repeat="intpProcess in getProgressInCurrentPage(intpProcesses | filter: nodeFilter)" >
+  <div id="{{intpProcess.properties.INTP_PROCESS_NAME}}">
+    <div class="row interpreter">
+      <div class="col-md-12">
+        <h3 class="interpreter-title">{{intpProcess.properties.INTP_PROCESS_NAME}}
+          <small ng-switch="intpProcess.properties.STATUS">
+            <small ng-switch-when="ONLINE">
+              <i style="color: green; margin-right: 3px;" class="fa fa-circle"
+                 uib-tooltip="online">
+              </i>
+            </small>
+            <small ng-switch-when="OFFLINE">
+              <i style="color: red; cursor: pointer" class="fa fa-circle"
+                 uib-tooltip="offline">
+              </i>
+            </small>
+          </small>
+        </h3>
+      </div>
+    </div>
+
+    <div class="row properties" style="margin-top:20px !important">
+      <div ng-show="_.isEmpty(intpProcess.properties)" class="col-md-12 gray40-message">
+        <em>Currently there are no properties set for this progress</em>
+      </div>
+      <div class="col-md-12" ng-show="!_.isEmpty(intpProcess.properties)">
+        <h5>Properties</h5>
+        <table class="table table-striped">
+          <thead>
+          <tr>
+            <th style="width:50%">name</th>
+            <th style="width:50%">value</th>
+          </tr>
+          </thead>
+          <tr ng-repeat="key in intpProcess.properties | sortByKey" >
+            <td style="vertical-align: middle;">{{key}}</td>
+            <td style="vertical-align: middle;">
+              <span editable-text="intpProcess.properties[key]" e-form="valueform" e-msd-elastic>
+                 {{intpProcess.properties[key]}}
+              </span>
+            </td>
+          </tr>
+        </table>
+      </div>
+    </div>
+  </div>
+</div>
+<!-- pagination -->
+<div class="cluster-pagination-container">
+  <ul uib-pagination class="pagination-sm"
+      total-items="filteredProcesses.length"
+      ng-model="pagination.currentPage"
+      items-per-page="pagination.itemsPerPage"
+      boundary-links="true" rotate="false"
+      max-size="pagination.maxPageCount"
+      previous-text="&lsaquo;" next-text="&rsaquo;"
+      first-text="&laquo;" last-text="&raquo;"></ul>
+</div>
diff --git a/zeppelin-web/src/components/navbar/navbar.controller.js b/zeppelin-web/src/components/navbar/navbar.controller.js
index 68a7f4a..d6b66fa 100644
--- a/zeppelin-web/src/components/navbar/navbar.controller.js
+++ b/zeppelin-web/src/components/navbar/navbar.controller.js
@@ -47,6 +47,16 @@ function NavCtrl($scope, $rootScope, $http, $routeParams, $location,
       });
   }
 
+  function getClusterAddr() {
+    $http.get(baseUrlSrv.getRestApiBase() + '/cluster/address').success(
+      function(data, status, headers, config) {
+        $rootScope.clusterAddr = data.body.clusterAddr;
+      }).error(
+      function(data, status, headers, config) {
+        console.log('Error %o %o', status, data.message);
+      });
+  }
+
   function initController() {
     $scope.isDrawNavbarNoteList = false;
     angular.element('#notebook-list').perfectScrollbar({suppressScrollX: true});
@@ -56,6 +66,7 @@ function NavCtrl($scope, $rootScope, $http, $routeParams, $location,
     });
 
     getZeppelinVersion();
+    getClusterAddr();
     loadNotes();
   }
 
diff --git a/zeppelin-web/src/components/navbar/navbar.html b/zeppelin-web/src/components/navbar/navbar.html
index 59d65c9..263ff06 100644
--- a/zeppelin-web/src/components/navbar/navbar.html
+++ b/zeppelin-web/src/components/navbar/navbar.html
@@ -102,6 +102,7 @@ limitations under the License.
               <li><a href="#/credential">Credential</a></li>
               <li><a href="#/helium">Helium</a></li>
               <li><a href="#/configuration">Configuration</a></li>
+              <li ng-if="clusterAddr !== ''"><a href="#/cluster">Cluster</a> </li>
               <li ng-if="ticket.principal && ticket.principal !== 'anonymous'" role="separator" style="margin: 5px 0;" class="divider"></li>
               <li ng-if="ticket.principal && ticket.principal !== 'anonymous'"><a ng-click="navbar.logout()">Logout</a></li>
             </ul>
diff --git a/zeppelin-web/src/index.js b/zeppelin-web/src/index.js
index 19d2d3e..1aee71f 100644
--- a/zeppelin-web/src/index.js
+++ b/zeppelin-web/src/index.js
@@ -36,6 +36,8 @@ import './app/visualization/builtins/visualization-linechart.js';
 import './app/visualization/builtins/visualization-scatterchart.js';
 
 import './app/jobmanager/jobmanager.component.js';
+import './app/cluster/cluster.controller.js';
+import './app/cluster/node.controller.js';
 import './app/interpreter/interpreter.controller.js';
 import './app/interpreter/interpreter.filter.js';
 import './app/interpreter/interpreter-item.directive.js';