You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2018/08/03 20:58:35 UTC

lucene-solr:branch_7x: SOLR-8207: Add "Nodes" view to the Admin UI "Cloud" tab, listing nodes and key metrics

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 79feed970 -> f97a28017


SOLR-8207: Add "Nodes" view to the Admin UI "Cloud" tab, listing nodes and key metrics

(cherry picked from commit 17a02c1)


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/f97a2801
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/f97a2801
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/f97a2801

Branch: refs/heads/branch_7x
Commit: f97a28017e952472e482951a3274e70a344cbf39
Parents: 79feed9
Author: Jan Høydahl <ja...@apache.org>
Authored: Fri Aug 3 13:55:11 2018 +0200
Committer: Jan Høydahl <ja...@apache.org>
Committed: Fri Aug 3 22:57:53 2018 +0200

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/core/CoreContainer.java     |   2 +-
 .../solr/handler/admin/AdminHandlersProxy.java  | 128 +++++
 .../solr/handler/admin/MetricsHandler.java      |  13 +-
 .../solr/handler/admin/SystemInfoHandler.java   |  11 +-
 .../handler/admin/AdminHandlersProxyTest.java   | 119 +++++
 .../solr/handler/admin/MetricsHandlerTest.java  |   8 +-
 solr/solr-ref-guide/src/cloud-screens.adoc      |  25 +-
 .../src/images/cloud-screens/cloud-nodes.png    | Bin 0 -> 185805 bytes
 .../apache/solr/common/params/CommonParams.java |   2 +
 solr/webapp/web/css/angular/cloud.css           | 193 ++++++++
 solr/webapp/web/css/angular/menu.css            |   2 +-
 solr/webapp/web/index.html                      |   4 +-
 solr/webapp/web/js/angular/controllers/cloud.js | 484 ++++++++++++++++++-
 solr/webapp/web/js/angular/services.js          |   7 +-
 solr/webapp/web/partials/cloud.html             | 132 +++++
 16 files changed, 1103 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 2702550..2b50f7d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -95,6 +95,8 @@ New Features
 
 * SOLR-12536: autoscaling policy support to equally distribute replicas on the basis of arbitrary properties (noble)
 
+* SOLR-8207: Add "Nodes" view to the Admin UI "Cloud" tab, listing nodes and key metrics (janhoy)
+
 * SOLR-11990: Make it possible to co-locate replicas of multiple collections together in a node. A collection may be
   co-located with another collection during collection creation time by specifying a 'withCollection' parameter. It can
   also be co-located afterwards by using the modify collection API. The co-location guarantee is enforced regardless of

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 440d2bd..67b8794 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -567,7 +567,7 @@ public class CoreContainer {
     configSetsHandler = createHandler(CONFIGSETS_HANDLER_PATH, cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
 
     // metricsHistoryHandler uses metricsHandler, so create it first
-    metricsHandler = new MetricsHandler(metricManager);
+    metricsHandler = new MetricsHandler(this);
     containerHandlers.put(METRICS_PATH, metricsHandler);
     metricsHandler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), metricTag, METRICS_PATH);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/core/src/java/org/apache/solr/handler/admin/AdminHandlersProxy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/AdminHandlersProxy.java b/solr/core/src/java/org/apache/solr/handler/admin/AdminHandlersProxy.java
new file mode 100644
index 0000000..ae3e01f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/AdminHandlersProxy.java
@@ -0,0 +1,128 @@
+/*
+ * 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.solr.handler.admin;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Pair;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Static methods to proxy calls to an Admin (GET) API to other nodes in the cluster and return a combined response
+ */
+public class AdminHandlersProxy {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final String PARAM_NODES = "nodes";
+
+  // Proxy this request to a different remote node if 'node' parameter is provided
+  public static boolean maybeProxyToNodes(SolrQueryRequest req, SolrQueryResponse rsp, CoreContainer container)
+      throws IOException, SolrServerException, InterruptedException {
+    String nodeNames = req.getParams().get(PARAM_NODES);
+    if (nodeNames == null || nodeNames.isEmpty()) {
+      return false; // local request
+    }
+
+    if (!container.isZooKeeperAware()) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Parameter " + PARAM_NODES + " only supported in Cloud mode");
+    }
+    
+    Set<String> nodes;
+    String pathStr = req.getPath();
+    
+    Map<String,String> paramsMap = req.getParams().toMap(new HashMap<>());
+    paramsMap.remove(PARAM_NODES);
+    SolrParams params = new MapSolrParams(paramsMap);
+    Set<String> liveNodes = container.getZkController().zkStateReader.getClusterState().getLiveNodes();
+    
+    if (nodeNames.equals("all")) {
+      nodes = liveNodes;
+      log.debug("All live nodes requested");
+    } else {
+      nodes = new HashSet<>(Arrays.asList(nodeNames.split(",")));
+      for (String nodeName : nodes) {
+        if (!nodeName.matches("^[^/:]+:\\d+_[\\w/]+$")) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Parameter " + PARAM_NODES + " has wrong format");
+        }
+
+        if (!liveNodes.contains(nodeName)) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Requested node " + nodeName + " is not part of cluster");
+        }
+      }       
+      log.debug("Nodes requested: {}", nodes);
+    }
+    log.debug(PARAM_NODES + " parameter {} specified on {} request", nodeNames, pathStr);
+    
+    Map<String, Pair<Future<NamedList<Object>>, SolrClient>> responses = new HashMap<>();
+    for (String node : nodes) {
+      responses.put(node, callRemoteNode(node, pathStr, params, container.getZkController()));
+    }
+    
+    for (Map.Entry<String, Pair<Future<NamedList<Object>>, SolrClient>> entry : responses.entrySet()) {
+      try {
+        NamedList<Object> resp = entry.getValue().first().get(10, TimeUnit.SECONDS);
+        entry.getValue().second().close();
+        rsp.add(entry.getKey(), resp);
+      } catch (ExecutionException ee) {
+        log.warn("Exception when fetching result from node {}", entry.getKey(), ee);
+      } catch (TimeoutException te) {
+        log.warn("Timeout when fetching result from node {}", entry.getKey(), te);
+      }
+    }
+    log.info("Fetched response from {} nodes: {}", responses.keySet().size(), responses.keySet());
+    return true;
+  } 
+
+  /**
+   * Makes a remote request and returns a future and the solr client. The caller is responsible for closing the client 
+   */
+  public static Pair<Future<NamedList<Object>>, SolrClient> callRemoteNode(String nodeName, String endpoint, 
+                                                                           SolrParams params, ZkController zkController) 
+      throws IOException, SolrServerException {
+    log.debug("Proxying {} request to node {}", endpoint, nodeName);
+    URL baseUrl = new URL(zkController.zkStateReader.getBaseUrlForNodeName(nodeName));
+    HttpSolrClient solr = new HttpSolrClient.Builder(baseUrl.toString()).build();
+    SolrRequest proxyReq = new GenericSolrRequest(SolrRequest.METHOD.GET, endpoint, params);
+    HttpSolrClient.HttpUriRequestResponse proxyResp = solr.httpUriRequest(proxyReq);
+    return new Pair<>(proxyResp.future, solr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
index ca291e8..1f1a820 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
@@ -41,6 +41,7 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.request.SolrQueryRequest;
@@ -67,11 +68,17 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName
   public static final String ALL = "all";
 
   private static final Pattern KEY_REGEX = Pattern.compile("(?<!" + Pattern.quote("\\") + ")" + Pattern.quote(":"));
+  private CoreContainer cc;
 
   public MetricsHandler() {
     this.metricManager = null;
   }
 
+  public MetricsHandler(CoreContainer coreContainer) {
+    this.metricManager = coreContainer.getMetricManager();
+    this.cc = coreContainer;
+  }
+
   public MetricsHandler(SolrMetricManager metricManager) {
     this.metricManager = metricManager;
   }
@@ -87,9 +94,13 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName
       throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "SolrMetricManager instance not initialized");
     }
 
+    if (cc != null && AdminHandlersProxy.maybeProxyToNodes(req, rsp, cc)) {
+      return; // Request was proxied to other node
+    }
+
     handleRequest(req.getParams(), (k, v) -> rsp.add(k, v));
   }
-
+  
   public void handleRequest(SolrParams params, BiConsumer<String, Object> consumer) throws Exception {
     boolean compact = params.getBool(COMPACT_PARAM, true);
     String[] keys = params.getParams(KEY_PARAM);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
index 8a2786c..d8e10ab 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
@@ -61,13 +61,14 @@ import static org.apache.solr.common.params.CommonParams.NAME;
 public class SystemInfoHandler extends RequestHandlerBase 
 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final String PARAM_NODE = "node";
 
   public static String REDACT_STRING = RedactionUtils.getRedactString();
 
   /**
    * <p>
    * Undocumented expert level system property to prevent doing a reverse lookup of our hostname.
-   * This property ill be logged as a suggested workaround if any probems are noticed when doing reverse 
+   * This property will be logged as a suggested workaround if any problems are noticed when doing reverse 
    * lookup.
    * </p>
    *
@@ -130,7 +131,11 @@ public class SystemInfoHandler extends RequestHandlerBase
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception
   {
+    rsp.setHttpCaching(false);
     SolrCore core = req.getCore();
+    if (AdminHandlersProxy.maybeProxyToNodes(req, rsp, getCoreContainer(req, core))) {
+      return; // Request was proxied to other node
+    }
     if (core != null) rsp.add( "core", getCoreInfo( core, req.getSchema() ) );
     boolean solrCloudMode =  getCoreContainer(req, core).isZooKeeperAware();
     rsp.add( "mode", solrCloudMode ? "solrcloud" : "std");
@@ -142,7 +147,9 @@ public class SystemInfoHandler extends RequestHandlerBase
     rsp.add( "lucene", getLuceneInfo() );
     rsp.add( "jvm", getJvmInfo() );
     rsp.add( "system", getSystemInfo() );
-    rsp.setHttpCaching(false);
+    if (solrCloudMode) {
+      rsp.add("node", getCoreContainer(req, core).getZkController().getNodeName());
+    }
   }
 
   private CoreContainer getCoreContainer(SolrQueryRequest req, SolrCore core) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/core/src/test/org/apache/solr/handler/admin/AdminHandlersProxyTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/AdminHandlersProxyTest.java b/solr/core/src/test/org/apache/solr/handler/admin/AdminHandlersProxyTest.java
new file mode 100644
index 0000000..93cbe3e
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/AdminHandlersProxyTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.solr.handler.admin;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.response.SimpleSolrResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class AdminHandlersProxyTest extends SolrCloudTestCase {
+  private CloseableHttpClient httpClient;
+  private CloudSolrClient solrClient;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    solrClient = getCloudSolrClient(cluster);
+    solrClient.connect(1000, TimeUnit.MILLISECONDS);
+    httpClient = (CloseableHttpClient) solrClient.getHttpClient();
+  }
+
+  @After
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    IOUtils.close(solrClient, httpClient);
+  }
+
+  @Test
+  public void proxySystemInfoHandlerAllNodes() throws IOException, SolrServerException {
+    MapSolrParams params = new MapSolrParams(Collections.singletonMap("nodes", "all"));
+    GenericSolrRequest req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/info/system", params);
+    SimpleSolrResponse rsp = req.process(solrClient, null);
+    NamedList<Object> nl = rsp.getResponse();
+    assertEquals(3, nl.size());
+    assertTrue(nl.getName(1).endsWith("_solr"));
+    assertTrue(nl.getName(2).endsWith("_solr"));
+    assertEquals("solrcloud", ((NamedList)nl.get(nl.getName(1))).get("mode"));
+    assertEquals(nl.getName(2), ((NamedList)nl.get(nl.getName(2))).get("node"));
+  }
+
+  @Test
+  public void proxyMetricsHandlerAllNodes() throws IOException, SolrServerException {
+    MapSolrParams params = new MapSolrParams(Collections.singletonMap("nodes", "all"));
+    GenericSolrRequest req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/metrics", params);
+    SimpleSolrResponse rsp = req.process(solrClient, null);
+    NamedList<Object> nl = rsp.getResponse();
+    assertEquals(3, nl.size());
+    assertTrue(nl.getName(1).endsWith("_solr"));
+    assertTrue(nl.getName(2).endsWith("_solr"));
+    assertNotNull(((NamedList)nl.get(nl.getName(1))).get("metrics"));
+  }
+
+  @Test(expected = SolrException.class)
+  public void proxySystemInfoHandlerNonExistingNode() throws IOException, SolrServerException {
+    MapSolrParams params = new MapSolrParams(Collections.singletonMap("nodes", "example.com:1234_solr"));
+    GenericSolrRequest req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/info/system", params);
+    SimpleSolrResponse rsp = req.process(solrClient, null);
+  }
+  
+  @Test
+  public void proxySystemInfoHandlerOneNode() {
+    Set<String> nodes = solrClient.getClusterStateProvider().getLiveNodes();
+    assertEquals(2, nodes.size());
+    nodes.forEach(node -> {
+      MapSolrParams params = new MapSolrParams(Collections.singletonMap("nodes", node));
+      GenericSolrRequest req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/admin/info/system", params);
+      SimpleSolrResponse rsp = null;
+      try {
+        rsp = req.process(solrClient, null);
+      } catch (Exception e) {
+        fail("Exception while proxying request to node " + node);
+      }
+      NamedList<Object> nl = rsp.getResponse();
+      assertEquals(2, nl.size());
+      assertEquals("solrcloud", ((NamedList)nl.get(nl.getName(1))).get("mode"));
+      assertEquals(nl.getName(1), ((NamedList)nl.get(nl.getName(1))).get("node"));
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
index 392bdfc..0fe5ad7 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
@@ -48,7 +48,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
 
   @Test
   public void test() throws Exception {
-    MetricsHandler handler = new MetricsHandler(h.getCoreContainer().getMetricManager());
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
 
     SolrQueryResponse resp = new SolrQueryResponse();
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", MetricsHandler.COMPACT_PARAM, "false", CommonParams.WT, "json"), resp);
@@ -179,7 +179,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
 
   @Test
   public void testCompact() throws Exception {
-    MetricsHandler handler = new MetricsHandler(h.getCoreContainer().getMetricManager());
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
 
     SolrQueryResponse resp = new SolrQueryResponse();
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", MetricsHandler.COMPACT_PARAM, "true"), resp);
@@ -197,7 +197,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
   public void testPropertyFilter() throws Exception {
     assertQ(req("*:*"), "//result[@numFound='0']");
 
-    MetricsHandler handler = new MetricsHandler(h.getCoreContainer().getMetricManager());
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
 
     SolrQueryResponse resp = new SolrQueryResponse();
     handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json",
@@ -234,7 +234,7 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
 
   @Test
   public void testKeyMetrics() throws Exception {
-    MetricsHandler handler = new MetricsHandler(h.getCoreContainer().getMetricManager());
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
 
     String key1 = "solr.core.collection1:CACHE.core.fieldCache";
     SolrQueryResponse resp = new SolrQueryResponse();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/solr-ref-guide/src/cloud-screens.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/cloud-screens.adoc b/solr/solr-ref-guide/src/cloud-screens.adoc
index 77cf9e5..34982ee 100644
--- a/solr/solr-ref-guide/src/cloud-screens.adoc
+++ b/solr/solr-ref-guide/src/cloud-screens.adoc
@@ -26,7 +26,24 @@ This screen provides status information about each collection & node in your clu
 The "Cloud" menu option is only available on Solr instances running in <<getting-started-with-solrcloud.adoc#getting-started-with-solrcloud,SolrCloud mode>>. Single node or master/slave replication instances of Solr will not display this option.
 ====
 
-Click on the Cloud option in the left-hand navigation, and a small sub-menu appears with options called "Tree", "Graph", and "Graph (Radial)". The default view ("Graph") shows a graph of each collection, the shards that make up those collections, and the addresses and type ("NRT", "TLOG" or "PULL") of each replica for each shard.
+Click on the "Cloud" option in the left-hand navigation, and a small sub-menu appears with options called "Nodes", "Tree", "Graph" and "Graph (Radial)". The sub-view selected by default is "Graph".
+
+== Nodes view
+The "Nodes" view shows a list of the hosts and nodes in the cluster along with key information for each: "CPU", "Heap", "Disk usage", "Requests", "Collections" and "Replicas".
+
+The example below shows the default "cloud" example with some documents added to the "gettingstarted" collection. Details are expanded for node on port 7574, showing more metadata and more metrics details. The screen provides links to navigate to nodes, collections and replicas. The table supports paging and filtering on host/node names and collection names.
+ 
+image::images/cloud-screens/cloud-nodes.png[image,width=900,height=415]
+
+== Tree view
+The "Tree" view shows a directory structure of the data in ZooKeeper, including cluster wide information regarding the `live_nodes` and `overseer` status, as well as collection specific information such as the `state.json`, current shard leaders, and configuration files in use. In this example, we see part of the `state.json`  definition for the "tlog" collection:
+
+image::images/cloud-screens/cloud-tree.png[image,width=487,height=250]
+
+As an aid to debugging, the data shown in the "Tree" view can be exported locally using the following command `bin/solr zk ls -r /`
+
+== Graph views
+The "Graph" view shows a graph of each collection, the shards that make up those collections, and the addresses and type ("NRT", "TLOG" or "PULL") of each replica for each shard.
 
 This example shows a simple cluster. In addition to the 2 shard, 2 replica "gettingstarted" collection, there is an additional "tlog" collection consisting of mixed TLOG and PULL replica types.
 
@@ -39,9 +56,3 @@ image::images/cloud-screens/cloud-hover.png[image,width=512,height=250]
 The "Graph (Radial)" option provides a different visual view of each node. Using the same example cluster, the radial graph view looks like:
 
 image::images/cloud-screens/cloud-radial.png[image,width=478,height=250]
-
-The "Tree" option shows a directory structure of the data in ZooKeeper, including cluster wide information regarding the `live_nodes` and `overseer` status, as well as collection specific information such as the `state.json`, current shard leaders, and configuration files in use. In this example, we see part of the `state.json`  definition for the "tlog" collection:
-
-image::images/cloud-screens/cloud-tree.png[image,width=487,height=250]
-
-As an aid to debugging, the data shown in the "Tree" view can be exported locally using the following command `bin/solr zk ls -r /`

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/solr-ref-guide/src/images/cloud-screens/cloud-nodes.png
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/images/cloud-screens/cloud-nodes.png b/solr/solr-ref-guide/src/images/cloud-screens/cloud-nodes.png
new file mode 100644
index 0000000..cecc758
Binary files /dev/null and b/solr/solr-ref-guide/src/images/cloud-screens/cloud-nodes.png differ

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
index d7bc1cb..600e479 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
@@ -181,6 +181,7 @@ public interface CommonParams {
   String AUTHZ_PATH = "/admin/authorization";
   String AUTHC_PATH = "/admin/authentication";
   String ZK_PATH = "/admin/zookeeper";
+  String SYSTEM_INFO_PATH = "/admin/info/system";
   String METRICS_PATH = "/admin/metrics";
   String METRICS_HISTORY_PATH = "/admin/metrics/history";
   String AUTOSCALING_PATH = "/admin/autoscaling";
@@ -192,6 +193,7 @@ public interface CommonParams {
       CORES_HANDLER_PATH,
       COLLECTIONS_HANDLER_PATH,
       CONFIGSETS_HANDLER_PATH,
+      SYSTEM_INFO_PATH,
       AUTHC_PATH,
       AUTHZ_PATH,
       METRICS_PATH,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/webapp/web/css/angular/cloud.css
----------------------------------------------------------------------
diff --git a/solr/webapp/web/css/angular/cloud.css b/solr/webapp/web/css/angular/cloud.css
index e7a16c1..c3d54a5 100644
--- a/solr/webapp/web/css/angular/cloud.css
+++ b/solr/webapp/web/css/angular/cloud.css
@@ -421,6 +421,12 @@ limitations under the License.
   padding-bottom: 15px;
 }
 
+#nodesPaging
+{
+  padding-top: 5px;
+  padding-bottom: 5px;
+}
+
 #content #cloud #legend .shard-inactive,
 #content #cloud #legend .shard-inactive li,
 #content #cloud #legend .shard-inactive li text,
@@ -449,3 +455,190 @@ limitations under the License.
 {
   fill: #007BA7;
 }
+
+/* Nodes tab */
+#nodes-table {
+  border-collapse: collapse;
+}
+
+#nodes-table td, #nodes-table th {
+  border: 1px solid #ddd;
+  padding: 8px;
+  vertical-align: top;
+}
+#nodes-table th {
+  font-weight: bolder;
+  font-stretch: extra-expanded;
+  background: #F8F8F8;
+}
+#content #cloud #nodes-content #nodes-table
+{
+  border-top: 1px solid #c0c0c0;
+  margin-top: 10px;
+  padding-top: 10px;
+}
+
+#content #cloud #nodes-content .host-name,
+#content #cloud #nodes-content .node-name a
+{
+  font-weight: bold;
+  font-size: larger;
+}
+
+#content #cloud #nodes-content a,
+#content #cloud #nodes-content a:hover,
+#content #cloud #nodes-content a.hover
+{
+  text-decoration: underline;
+  text-decoration-style: dotted;
+  text-decoration-color: #beebff;
+}
+
+#content #cloud #nodes-content a:hover,
+#content #cloud #nodes-content a.hover
+{
+  background-color: #beebff;
+}
+
+#content #cloud #nodes-content .host-spec,
+#content #cloud #nodes-content .node-spec,
+#content #cloud #nodes-content .node-spec a
+{
+  font-style: italic;
+}
+#content #cloud #nodes-content .node-uptime
+{
+  font-weight: bolder;
+  font-size: 20px;
+}
+#content #cloud #nodes-content .node-load,
+#content #cloud #nodes-content .node-cpu,
+#content #cloud #nodes-content .node-heap,
+#content #cloud #nodes-content .node-disk
+{
+  font-weight: bolder;
+  font-size: 20px;
+}
+
+#content #cloud #nodes-content .pct-normal
+{
+  color: darkgreen;
+}
+
+#content #cloud #nodes-content .pct-warn
+{
+  color: orange;
+}
+
+#content #cloud #nodes-content .pct-critical
+{
+  color: red;
+}
+
+/* Styling of reload and details buttons */
+#content #cloud #controls
+{
+  display: block;
+  height: 30px;
+}
+
+#content #cloud .reload
+{
+  background-image: url( ../../img/ico/arrow-circle.png );
+  padding-left: 21px;
+  float: left;
+}
+
+#content #cloud .reload.loader
+{
+  padding-left: 0;
+}
+
+#content #cloud .details-button 
+{
+  background-image: url(../../img/ico/ui-check-box-uncheck.png);
+  background-position: 0 50%;
+  color: #8D8D8D;
+  margin-top: 7px;
+  margin-left: 10px;
+  padding-left: 21px;
+  width: 30px;
+}
+
+#content #cloud .details-button.on
+{
+  background-image: url( ../../img/ico/ui-check-box.png );
+  color: #333;
+}
+
+#content #cloud #nodes-content .more
+{
+  font-style: italic;
+  text-underline: #0000fa;
+}
+
+/* Disk usage details d3 chart bars style */
+.chart {
+    background: #eee;
+    padding: 1px;
+}
+.chart div {
+    width:90%;
+}
+.chart div div {
+    display:inline-block;
+}
+.chart div div.rect {
+    transition: all 0.5s ease-out;
+    -moz-transition: all 0.5s ease-out;
+    -webkit-transition: all 0.5s ease-out;
+    width:0;
+    font: 10px sans-serif;
+    background-color: #4CAF50;
+    text-align: left;
+    padding: 3px;
+    margin: 2px;
+    color: #000000;
+    box-shadow: 1px 1px 1px #666;
+}
+
+#content #nodes-content .leader
+{
+  font-weight: bold;
+}
+
+#content #nodes-content .scroll-height-250
+{
+  max-height: 250px;
+  overflow-scrolling: auto;
+  overflow: auto;
+  /*overflow-y: auto;*/
+}
+
+#content #nodes-content .min-width-150
+{
+  min-width: 150px;
+}
+
+#content #cloud #nodes-content .node-cores
+{
+  min-width: 150px;
+}
+
+#content #nodes-content .core-details
+{
+  padding-left: 21px;
+}
+
+
+
+::-webkit-scrollbar {
+    -webkit-appearance: none;
+    width: 7px;
+}
+
+::-webkit-scrollbar-thumb {
+    border-radius: 4px;
+    background-color: rgba(0,0,0,.5);
+    -webkit-box-shadow: 0 0 1px rgba(255,255,255,.5);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/webapp/web/css/angular/menu.css
----------------------------------------------------------------------
diff --git a/solr/webapp/web/css/angular/menu.css b/solr/webapp/web/css/angular/menu.css
index 549d737..ba5e0b6 100644
--- a/solr/webapp/web/css/angular/menu.css
+++ b/solr/webapp/web/css/angular/menu.css
@@ -260,9 +260,9 @@ limitations under the License.
 
 #menu #cloud.global p a { background-image: url( ../../img/ico/network-cloud.png ); }
 #menu #cloud.global .tree a { background-image: url( ../../img/ico/folder-tree.png ); }
+#menu #cloud.global .nodes a { background-image: url( ../../img/solr-ico.png ); }
 #menu #cloud.global .graph a { background-image: url( ../../img/ico/molecule.png ); }
 #menu #cloud.global .rgraph a { background-image: url( ../../img/ico/asterisk.png ); }
-#menu #cloud.global .dump a { background-image: url( ../../img/ico/download-cloud.png ); }
 
 .sub-menu .ping.error a
 {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/webapp/web/index.html
----------------------------------------------------------------------
diff --git a/solr/webapp/web/index.html b/solr/webapp/web/index.html
index 3b4fb62..256af89 100644
--- a/solr/webapp/web/index.html
+++ b/solr/webapp/web/index.html
@@ -150,10 +150,10 @@ limitations under the License.
 
             <li id="cloud" class="global optional" ng-show="isCloudEnabled" ng-class="{active:showingCloud}"><p><a href="#/~cloud">Cloud</a></p>
               <ul ng-show="showingCloud">
+                <li class="nodes" ng-class="{active:page=='cloud-nodes'}"><a href="#/~cloud?view=nodes">Nodes</a></li>
                 <li class="tree" ng-class="{active:page=='cloud-tree'}"><a href="#/~cloud?view=tree">Tree</a></li>
-                <li class="graph" ng-class="{active:page=='cloud-graph'}"><a href="#/~cloud">Graph</a></li>
+                <li class="graph" ng-class="{active:page=='cloud-graph'}"><a href="#/~cloud?view=graph">Graph</a></li>
                 <li class="rgraph" ng-class="{active:page=='cloud-rgraph'}"><a href="#/~cloud?view=rgraph">Graph (Radial)</a></li>
-                <!--<li class="dump" ng-class="{active:page=='cloud-dump'}"><a ng-click="dumpCloud()">Dump</a></li>-->
               </ul>
             </li>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/webapp/web/js/angular/controllers/cloud.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/controllers/cloud.js b/solr/webapp/web/js/angular/controllers/cloud.js
index f1af1f7..08eea38 100644
--- a/solr/webapp/web/js/angular/controllers/cloud.js
+++ b/solr/webapp/web/js/angular/controllers/cloud.js
@@ -16,7 +16,7 @@
 */
 
 solrAdminApp.controller('CloudController',
-    function($scope, $location, Zookeeper, Constants) {
+    function($scope, $location, Zookeeper, Constants, Collections, System, Metrics) {
 
         $scope.showDebug = false;
 
@@ -26,22 +26,466 @@ solrAdminApp.controller('CloudController',
 
         $scope.closeDebug = function() {
             $scope.showDebug = false;
-        }
+        };
 
         var view = $location.search().view ? $location.search().view : "graph";
-        if (view == "tree") {
+        if (view === "tree") {
             $scope.resetMenu("cloud-tree", Constants.IS_ROOT_PAGE);
             treeSubController($scope, Zookeeper);
-        } else if (view == "rgraph") {
+        } else if (view === "rgraph") {
             $scope.resetMenu("cloud-rgraph", Constants.IS_ROOT_PAGE);
             graphSubController($scope, Zookeeper, true);
-        } else if (view == "graph") {
+        } else if (view === "graph") {
             $scope.resetMenu("cloud-graph", Constants.IS_ROOT_PAGE);
             graphSubController($scope, Zookeeper, false);
+        } else if (view === "nodes") {
+            $scope.resetMenu("cloud-nodes", Constants.IS_ROOT_PAGE);
+            nodesSubController($scope, Collections, System, Metrics);
         }
     }
 );
 
+function getOrCreateObj(name, object) {
+  if (name in object) {
+    entry = object[name];
+  } else {
+    entry = {};
+    object[name] = entry;
+  }
+  return entry;
+}
+
+function getOrCreateList(name, object) {
+  if (name in object) {
+    entry = object[name];
+  } else {
+    entry = [];
+    object[name] = entry;
+  }
+  return entry;
+}
+
+function ensureInList(string, list) {
+  if (list.indexOf(string) === -1) {
+    list.push(string);
+  }
+}
+
+/* Puts a node name into the hosts structure */
+function ensureNodeInHosts(node_name, hosts) {
+  var hostName = node_name.split(":")[0];
+  var host = getOrCreateObj(hostName, hosts);
+  var hostNodes = getOrCreateList("nodes", host);
+  ensureInList(node_name, hostNodes);
+}
+
+// from http://scratch99.com/web-development/javascript/convert-bytes-to-mb-kb/
+function bytesToSize(bytes) {
+  var sizes = ['b', 'Kb', 'Mb', 'Gb', 'Tb'];
+  if (bytes === 0) return '0b';
+  var i = parseInt(Math.floor(Math.log(bytes) / Math.log(1024)));
+  if (bytes === 0) return bytes + '' + sizes[i];
+  return (bytes / Math.pow(1024, i)).toFixed(1) + '' + sizes[i];
+}
+
+function numDocsHuman(docs) {
+  var sizes = ['', 'k', 'mn', 'bn', 'tn'];
+  if (docs === 0) return '0';
+  var i = parseInt(Math.floor(Math.log(docs) / Math.log(1000)));
+  if (i === 0) return docs + '' + sizes[i];
+  return (docs / Math.pow(1000, i)).toFixed(1) + '' + sizes[i];
+}
+
+/* Returns a style class depending on percentage */
+var styleForPct = function (pct) {
+  if (pct < 60) return "pct-normal";
+  if (pct < 80) return "pct-warn";
+  return "pct-critical"
+};
+
+function isNumeric(n) {
+  return !isNaN(parseFloat(n)) && isFinite(n);
+}
+
+var nodesSubController = function($scope, Collections, System, Metrics) {
+  $scope.pageSize = 10;
+  $scope.showNodes = true;
+  $scope.showTree = false;
+  $scope.showGraph = false;
+  $scope.showData = false;
+  $scope.showAllDetails = false;
+  $scope.showDetails = {};
+  $scope.from = 0;
+  $scope.to = $scope.pageSize - 1;
+  $scope.filterType = "node"; // Pre-initialize dropdown
+
+  $scope.toggleAllDetails = function() {
+    $scope.showAllDetails = !$scope.showAllDetails;
+    for (var node in $scope.nodes) {
+      $scope.showDetails[node] = $scope.showAllDetails;
+    }
+    for (var host in $scope.hosts) {
+      $scope.showDetails[host] = $scope.showAllDetails;
+    }
+  };
+
+  $scope.toggleDetails = function(key) {
+    $scope.showDetails[key] = !$scope.showDetails[key] === true;
+  };
+
+  $scope.toggleHostDetails = function(key) {
+    $scope.showDetails[key] = !$scope.showDetails[key] === true;
+    for (var nodeId in $scope.hosts[key].nodes) {
+      var node = $scope.hosts[key].nodes[nodeId];
+      $scope.showDetails[node] = $scope.showDetails[key];
+    }
+  };
+
+  $scope.nextPage = function() {
+    $scope.from += parseInt($scope.pageSize);
+    $scope.reload();
+  };
+
+  $scope.previousPage = function() {
+    $scope.from = Math.max(0, $scope.from - parseInt($scope.pageSize));
+    $scope.reload();
+  };
+  
+  // Checks if this node is the first (alphabetically) for a given host. Used to decide rowspan in table
+  $scope.isFirstNodeForHost = function(node) {
+    var hostName = node.split(":")[0]; 
+    var nodesInHost = $scope.filteredNodes.filter(function (node) {
+      return node.startsWith(hostName);
+    });
+    return nodesInHost[0] === node;
+  };
+  
+  // Initializes the cluster state, list of nodes, collections etc
+  $scope.initClusterState = function() {
+    var nodes = {};
+    var hosts = {};
+    var live_nodes = [];
+
+    // We build a node-centric view of the cluster state which we can easily consume to render the table
+    Collections.status(function (data) {
+      // Fetch cluster state from collections API and invert to a nodes structure
+      for (var name in data.cluster.collections) {
+        var collection = data.cluster.collections[name];
+        collection.name = name;
+        var shards = collection.shards;
+        collection.shards = [];
+        for (var shardName in shards) {
+          var shard = shards[shardName];
+          shard.name = shardName;
+          shard.collection = collection.name;
+          var replicas = shard.replicas;
+          shard.replicas = [];
+          for (var replicaName in replicas) {
+            var core = replicas[replicaName];
+            core.name = replicaName;
+            core.collection = collection.name;
+            core.shard = shard.name;
+            core.shard_state = shard.state;
+
+            var node_name = core['node_name'];
+            var node = getOrCreateObj(node_name, nodes);
+            var cores = getOrCreateList("cores", node);
+            cores.push(core);
+            node['base_url'] = core.base_url;
+            node['id'] = core.base_url.replace(/[^\w\d]/g, '');
+            node['host'] = node_name.split(":")[0];
+            var collections = getOrCreateList("collections", node);
+            ensureInList(core.collection, collections);
+            ensureNodeInHosts(node_name, hosts);
+          }
+        }
+      }
+
+      live_nodes = data.cluster.live_nodes;
+      for (n in data.cluster.live_nodes) {
+        node = data.cluster.live_nodes[n];
+        if (!(node in nodes)) {
+          var hostName = node.split(":")[0];
+          nodes[node] = {};
+          nodes[node]['host'] = hostName;
+        }
+        ensureNodeInHosts(node, hosts);
+      }
+
+      // Make sure nodes are sorted alphabetically to align with rowspan in table 
+      for (var host in hosts) {
+        hosts[host].nodes.sort();
+      }
+
+      $scope.nodes = nodes;
+      $scope.hosts = hosts;
+      $scope.live_nodes = live_nodes;
+
+      $scope.Math = window.Math;
+      $scope.reload();
+    });
+  };
+
+  $scope.filterInput = function() {
+    $scope.from = 0;
+    $scope.to = $scope.pageSize - 1;
+    $scope.reload();
+  };
+
+  /*
+    Reload will fetch data for the current page of the table and thus refresh numbers.
+    It is also called whenever a filter or paging action is executed 
+   */
+  $scope.reload = function() {
+    var nodes = $scope.nodes;
+    var node_keys = Object.keys(nodes);
+    var hosts = $scope.hosts;
+    var live_nodes = $scope.live_nodes;
+    var hostNames = Object.keys(hosts);
+    hostNames.sort();
+    var pageSize = isNumeric($scope.pageSize) ? $scope.pageSize : 10;
+
+    // Calculate what nodes that will show on this page
+    var nodesToShow = [];
+    var nodesParam;
+    var hostsToShow = [];
+    var filteredNodes;
+    var filteredHosts;
+    var isFiltered = false;
+    switch ($scope.filterType) {
+      case "node":  // Find what nodes match the node filter
+        if ($scope.nodeFilter) {
+          filteredNodes = node_keys.filter(function (node) {
+            return node.indexOf($scope.nodeFilter) !== -1;
+          });
+        }
+        break;
+
+      case "collection": // Find what collections match the collection filter and what nodes that have these collections
+        if ($scope.collectionFilter) {
+          candidateNodes = {};
+          nodesCollections = [];
+          for (var i = 0 ; i < node_keys.length ; i++) {
+            var node_name = node_keys[i];
+            var node = nodes[node_name];
+            nodeColl = {};
+            nodeColl['node'] = node_name;
+            collections = {};
+            node.cores.forEach(function(core) {
+              collections[core.collection] = true;
+            });
+            nodeColl['collections'] = Object.keys(collections);
+            nodesCollections.push(nodeColl);
+          }
+          nodesCollections.forEach(function(nc) {
+            matchingColls = nc['collections'].filter(function (collection) {
+              return collection.indexOf($scope.collectionFilter) !== -1;
+            });
+            if (matchingColls.length > 0) {
+              candidateNodes[nc.node] = true;
+            }
+          });
+          filteredNodes = Object.keys(candidateNodes);
+        }
+        break;
+
+      case "health":
+
+    }
+    
+    if (filteredNodes) {
+      // If filtering is active, calculate what hosts contain the nodes that match the filters
+      isFiltered = true;
+      filteredHosts = filteredNodes.map(function (node) {
+        return node.split(":")[0];
+      }).filter(function (item, index, self) {
+        return self.indexOf(item) === index;
+      });
+    } else {
+      filteredNodes = node_keys;
+      filteredHosts = hostNames;
+    }
+    filteredNodes.sort();
+    filteredHosts.sort();
+    
+    // Find what hosts & nodes (from the filtered set) that should be displayed on current page
+    for (var id = $scope.from ; id < $scope.from + pageSize && filteredHosts[id] ; id++) {
+      var hostName = filteredHosts[id];
+      hostsToShow.push(hostName);
+      if (isFiltered) { // Only show the nodes per host matching active filter
+        nodesToShow = nodesToShow.concat(filteredNodes.filter(function (node) {
+          return node.startsWith(hostName);
+        }));
+      } else {
+        nodesToShow = nodesToShow.concat(hosts[hostName]['nodes']);
+      }
+    }
+    nodesParam = nodesToShow.join(',');
+    $scope.nextEnabled = $scope.from + pageSize < filteredHosts.length;
+    $scope.prevEnabled = $scope.from - pageSize >= 0;
+    nodesToShow.sort();
+    hostsToShow.sort();
+
+    /*
+     Fetch system info for all selected nodes
+     Pick the data we want to display and add it to the node-centric data structure
+      */
+    System.get({"nodes": nodesParam}, function (systemResponse) {
+      for (var node in systemResponse) {
+        if (node in nodes) {
+          var s = systemResponse[node];
+          nodes[node]['system'] = s;
+          var memTotal = s.system.totalPhysicalMemorySize;
+          var memFree = s.system.freePhysicalMemorySize;
+          var memPercentage = Math.floor((memTotal - memFree) / memTotal * 100);
+          nodes[node]['memUsedPct'] = memPercentage;
+          nodes[node]['memUsedPctStyle'] = styleForPct(memPercentage);
+          nodes[node]['memTotal'] = bytesToSize(memTotal);
+          nodes[node]['memFree'] = bytesToSize(memFree);
+          nodes[node]['memUsed'] = bytesToSize(memTotal - memFree);
+
+          var heapTotal = s.jvm.memory.raw.total;
+          var heapFree = s.jvm.memory.raw.free;
+          var heapPercentage = Math.floor((heapTotal - heapFree) / heapTotal * 100);
+          nodes[node]['heapUsed'] = bytesToSize(heapTotal - heapFree);
+          nodes[node]['heapUsedPct'] = heapPercentage;
+          nodes[node]['heapUsedPctStyle'] = styleForPct(heapPercentage);
+          nodes[node]['heapTotal'] = bytesToSize(heapTotal);
+          nodes[node]['heapFree'] = bytesToSize(heapFree);
+
+          var jvmUptime = s.jvm.jmx.upTimeMS / 1000; // Seconds
+          nodes[node]['jvmUptime'] = secondsForHumans(jvmUptime);
+          nodes[node]['jvmUptimeSec'] = jvmUptime;
+
+          nodes[node]['uptime'] = s.system.uptime.replace(/.*up (.*?,.*?),.*/, "$1");
+          nodes[node]['loadAvg'] = Math.round(s.system.systemLoadAverage * 100) / 100;
+          nodes[node]['cpuPct'] = Math.ceil(s.system.processCpuLoad);
+          nodes[node]['cpuPctStyle'] = styleForPct(Math.ceil(s.system.processCpuLoad));
+          nodes[node]['maxFileDescriptorCount'] = s.system.maxFileDescriptorCount;
+          nodes[node]['openFileDescriptorCount'] = s.system.openFileDescriptorCount;
+        }
+      }
+    });
+
+    /*
+     Fetch metrics for all selected nodes. Only pull the metrics that we'll show to save bandwidth
+     Pick the data we want to display and add it to the node-centric data structure
+      */
+    Metrics.get({
+          "nodes": nodesParam,
+          "prefix": "CONTAINER.fs,org.eclipse.jetty.server.handler.DefaultHandler.get-requests,INDEX.sizeInBytes,SEARCHER.searcher.numDocs,SEARCHER.searcher.deletedDocs,SEARCHER.searcher.warmupTime"
+        },
+        function (metricsResponse) {
+          for (var node in metricsResponse) {
+            if (node in nodes) {
+              var m = metricsResponse[node];
+              nodes[node]['metrics'] = m;
+              var diskTotal = m.metrics['solr.node']['CONTAINER.fs.totalSpace'];
+              var diskFree = m.metrics['solr.node']['CONTAINER.fs.usableSpace'];
+              var diskPercentage = Math.floor((diskTotal - diskFree) / diskTotal * 100);
+              nodes[node]['diskUsedPct'] = diskPercentage;
+              nodes[node]['diskUsedPctStyle'] = styleForPct(diskPercentage);
+              nodes[node]['diskTotal'] = bytesToSize(diskTotal);
+              nodes[node]['diskFree'] = bytesToSize(diskFree);
+
+              var r = m.metrics['solr.jetty']['org.eclipse.jetty.server.handler.DefaultHandler.get-requests'];
+              nodes[node]['req'] = r.count;
+              nodes[node]['req1minRate'] = Math.floor(r['1minRate'] * 100) / 100;
+              nodes[node]['req5minRate'] = Math.floor(r['5minRate'] * 100) / 100;
+              nodes[node]['req15minRate'] = Math.floor(r['15minRate'] * 100) / 100;
+              nodes[node]['reqp75_ms'] = Math.floor(r['p75_ms']);
+              nodes[node]['reqp95_ms'] = Math.floor(r['p95_ms']);
+              nodes[node]['reqp99_ms'] = Math.floor(r['p99_ms']);
+
+              var cores = nodes[node]['cores'];
+              var indexSizeTotal = 0;
+              var docsTotal = 0;
+              var graphData = [];
+              if (cores) {
+                for (coreId in cores) {
+                  var core = cores[coreId];
+                  var keyName = "solr.core." + core['core'].replace('_', '.').replace('_', '.');
+                  var nodeMetric = m.metrics[keyName];
+                  var size = nodeMetric['INDEX.sizeInBytes'];
+                  size = (typeof size !== 'undefined') ? size : 0;
+                  core['sizeInBytes'] = size;
+                  core['size'] = bytesToSize(size);
+                  core['label'] = core['core'].replace('_shard', '_s').replace(/_replica_./, 'r');
+                  indexSizeTotal += size;
+                  var numDocs = nodeMetric['SEARCHER.searcher.numDocs'];
+                  numDocs = (typeof numDocs !== 'undefined') ? numDocs : 0;
+                  core['numDocs'] = numDocs;
+                  core['numDocsHuman'] = numDocsHuman(numDocs);
+                  core['avgSizePerDoc'] = bytesToSize(numDocs === 0 ? 0 : size / numDocs);
+                  var deletedDocs = nodeMetric['SEARCHER.searcher.deletedDocs'];
+                  deletedDocs = (typeof deletedDocs !== 'undefined') ? deletedDocs : 0;
+                  core['deletedDocs'] = deletedDocs;
+                  core['deletedDocsHuman'] = numDocsHuman(deletedDocs);
+                  var warmupTime = nodeMetric['SEARCHER.searcher.warmupTime'];
+                  warmupTime = (typeof warmupTime !== 'undefined') ? warmupTime : 0;
+                  core['warmupTime'] = warmupTime;
+                  docsTotal += core['numDocs'];
+                }
+                for (coreId in cores) {
+                  core = cores[coreId];
+                  var graphObj = {};
+                  graphObj['label'] = core['label'];
+                  graphObj['size'] = core['sizeInBytes'];
+                  graphObj['sizeHuman'] = core['size'];
+                  graphObj['pct'] = (core['sizeInBytes'] / indexSizeTotal) * 100;
+                  graphData.push(graphObj);
+                }
+                cores.sort(function (a, b) {
+                  return b.sizeInBytes - a.sizeInBytes
+                });
+              } else {
+                cores = {};
+              }
+              graphData.sort(function (a, b) {
+                return b.size - a.size
+              });
+              nodes[node]['graphData'] = graphData;
+              nodes[node]['numDocs'] = numDocsHuman(docsTotal);
+              nodes[node]['sizeInBytes'] = indexSizeTotal;
+              nodes[node]['size'] = bytesToSize(indexSizeTotal);
+              nodes[node]['sizePerDoc'] = docsTotal === 0 ? '0b' : bytesToSize(indexSizeTotal / docsTotal);
+
+              // Build the d3 powered bar chart
+              $('#chart' + nodes[node]['id']).empty();
+              var chart = d3.select('#chart' + nodes[node]['id']).append('div').attr('class', 'chart');
+
+              // Add one div per bar which will group together both labels and bars
+              var g = chart.selectAll('div')
+                  .data(nodes[node]['graphData']).enter()
+                  .append('div');
+
+              // Add the bars
+              var bars = g.append("div")
+                  .attr("class", "rect")
+                  .text(function (d) {
+                    return d.label + ':\u00A0\u00A0' + d.sizeHuman;
+                  });
+
+              // Execute the transition to show the bars
+              bars.transition()
+                  .ease('elastic')
+                  .style('width', function (d) {
+                    return d.pct + '%';
+                  });
+            }
+          }
+        });
+    $scope.nodes = nodes;
+    $scope.hosts = hosts;
+    $scope.live_nodes = live_nodes;
+    $scope.nodesToShow = nodesToShow;
+    $scope.hostsToShow = hostsToShow;
+    $scope.filteredNodes = filteredNodes;
+    $scope.filteredHosts = filteredHosts;
+  };
+  $scope.initClusterState();
+};
+
 var treeSubController = function($scope, Zookeeper) {
     $scope.showTree = true;
     $scope.showGraph = false;
@@ -78,6 +522,28 @@ var treeSubController = function($scope, Zookeeper) {
     $scope.initTree();
 };
 
+/**
+ * Translates seconds into human readable format of seconds, minutes, hours, days, and years
+ * 
+ * @param  {number} seconds The number of seconds to be processed
+ * @return {string}         The phrase describing the the amount of time
+ */
+function secondsForHumans ( seconds ) {
+    var levels = [
+        [Math.floor(seconds / 31536000), 'y'],
+        [Math.floor((seconds % 31536000) / 86400), 'd'],
+        [Math.floor(((seconds % 31536000) % 86400) / 3600), 'h'],
+        [Math.floor((((seconds % 31536000) % 86400) % 3600) / 60), 'm']
+    ];
+    var returntext = '';
+
+    for (var i = 0, max = levels.length; i < max; i++) {
+        if ( levels[i][0] === 0 ) continue;
+        returntext += ' ' + levels[i][0] + levels[i][1];
+    }
+    return returntext.trim() === '' ? '0m' : returntext.trim();
+}
+
 var graphSubController = function ($scope, Zookeeper, isRadial) {
     $scope.showTree = false;
     $scope.showGraph = true;
@@ -101,17 +567,17 @@ var graphSubController = function ($scope, Zookeeper, isRadial) {
     $scope.next = function() {
         $scope.pos += $scope.rows;
         $scope.initGraph();
-    }
+    };
 
     $scope.previous = function() {
         $scope.pos = Math.max(0, $scope.pos - $scope.rows);
         $scope.initGraph();
-    }
+    };
 
     $scope.resetGraph = function() {
         $scope.pos = 0;
         $scope.initGraph();
-    }
+    };
 
     $scope.initGraph = function() {
         Zookeeper.liveNodes(function (data) {
@@ -514,4 +980,4 @@ solrAdminApp.directive('graph', function(Constants) {
             }
         }
     };
-})
+});

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/webapp/web/js/angular/services.js
----------------------------------------------------------------------
diff --git a/solr/webapp/web/js/angular/services.js b/solr/webapp/web/js/angular/services.js
index 4d89347..66f2654 100644
--- a/solr/webapp/web/js/angular/services.js
+++ b/solr/webapp/web/js/angular/services.js
@@ -19,8 +19,12 @@ var solrAdminServices = angular.module('solrAdminServices', ['ngResource']);
 
 solrAdminServices.factory('System',
   ['$resource', function($resource) {
-    return $resource('admin/info/system', {"wt":"json", "_":Date.now()});
+    return $resource('admin/info/system', {"wt":"json", "nodes": "@nodes", "_":Date.now()});
   }])
+.factory('Metrics',
+    ['$resource', function($resource) {
+      return $resource('admin/metrics', {"wt":"json", "nodes": "@nodes", "prefix":"@prefix", "_":Date.now()});
+    }])
 .factory('Collections',
   ['$resource', function($resource) {
     return $resource('admin/collections',
@@ -63,7 +67,6 @@ solrAdminServices.factory('System',
   ['$resource', function($resource) {
     return $resource('admin/zookeeper', {wt:'json', _:Date.now()}, {
       "simple": {},
-      "dump": {params: {dump: "true"}},
       "liveNodes": {params: {path: '/live_nodes'}},
       "clusterState": {params: {detail: "true", path: "/clusterstate.json"}},
       "detail": {params: {detail: "true", path: "@path"}},

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f97a2801/solr/webapp/web/partials/cloud.html
----------------------------------------------------------------------
diff --git a/solr/webapp/web/partials/cloud.html b/solr/webapp/web/partials/cloud.html
index c13ccc6..078c9af 100644
--- a/solr/webapp/web/partials/cloud.html
+++ b/solr/webapp/web/partials/cloud.html
@@ -48,6 +48,138 @@ limitations under the License.
 
     </div>
 
+    <div id="nodes-content" class="content clearfix" ng-show="showNodes">
+      <div id="controls">
+        <a class="reload" ng-click="initClusterState()"><span>Refresh</span></a>
+        <a class="details-button" ng-click="toggleAllDetails()" ng-class="{on:showAllDetails}">
+          <span>Show all details</span>
+        </a>
+      </div>
+      <div>
+        <div id="nodesPaging">
+         <button ng-show="prevEnabled" ng-click="previousPage()" id="nodesPagingPrev">&lt; Previous</button>
+         <span ng-show="filteredHosts.length==0">No hosts found.</span>
+         <span ng-hide="filteredHosts.length==0">Hosts {{from+1}} - {{from + hostsToShow.length}} of {{filteredHosts.length}}.</span>&nbsp;
+
+         Filter by:&nbsp;<select ng-model="filterType" ng-change="filterInput()" id="nodesPagingFilterType">
+           <option value="node">Host/node name</option>
+           <option value="collection">Collection name</option>
+         </select>&nbsp;
+
+         <span ng-show="filterType=='node'">
+           <input ng-model="nodeFilter" type="text" size="10" name="nodefilter" ng-change="filterInput()" ng-model-options='{ debounce: 500 }' />&nbsp;
+         </span>
+          <span ng-show="filterType=='collection'">
+            <input ng-model="collectionFilter" type="text" size="10" name="collectionfilter" ng-change="filterInput()" ng-model-options='{ debounce: 500 }'/>&nbsp;
+          </span>
+         Show <input ng-model="pageSize" ng-change="filterInput()" type="text" size="2" name="rows" ng-model-options='{ debounce: 500 }'/> hosts per page.
+         <button ng-show="nextEnabled" ng-click="nextPage()">Next &gt;</button>
+        </div>
+      </div>
+      <table id="nodes-table">
+        <thead>
+          <tr>
+              <th>Host</th>
+              <th>Node</th>
+              <th>CPU</th>
+              <th>Heap</th>
+              <th>Disk usage</th>
+              <th>Requests</th>
+              <th>Collections</th>
+              <th>Replicas</th>
+          </tr>
+        </thead>
+        <tbody>
+          <tr ng-repeat="key in nodesToShow | orderBy:'key.order'" ng-init="n = nodes[key]">
+              <td rowspan="{{hosts[n.host].nodes.length}}" ng-show="isFirstNodeForHost(key)">
+                <div class="host-name">{{n.host}}</div>
+                <span class="host-spec" ng-show="!showDetails[n.host]">
+                  <span title="{{n.system.system.uptime}}">{{n.system.system.name}}</span> 
+                  <span title="free: {{n.memFree}}">{{n.memTotal}}</span>
+                  <span title="{{n.system.jvm.name}} {{n.system.jvm.version}}">Java {{n.system.jvm.spec.version}}</span>
+                  <br/>Load: {{n.loadAvg}}
+                </span>
+                <div class="host-spec" ng-show="showDetails[n.host]">
+                  {{n.system.system.name}} {{n.system.system.version}}, {{n.system.system.availableProcessors}}cpu<br/>
+                  Uptime: {{n.uptime}}<br/>
+                  <span title="Used: {{n.memUsed}} - includes OS file-cache, and it is normal for it to approach 100%">Memory: {{n.memTotal}}</span><br/> 
+                  File descriptors: {{n.openFileDescriptorCount}}/{{n.maxFileDescriptorCount}}<br/>
+                  Disk: <span class="{{n.diskUsedPctStyle}}" title="Nodes may use other disks too">{{n.diskTotal}} used: {{n.diskUsedPct}}%</span><br/>
+                  Load: {{n.loadAvg}}
+                </div>
+                <div class="node-spec" ng-click="toggleHostDetails(n.host)">
+                  <a ng-show="showDetails[n.host]">hide details...</a>
+                  <a ng-show="!showDetails[n.host]">show details...</a>
+                </div>
+              </td>
+              <td><div class="node-name"><a href="{{n.base_url}}">{{key.replace(n.host+':', '')}}</a></div>
+                Uptime: {{n.jvmUptime}}<br/>
+                <div class="node-spec" ng-show="showDetails[key]">
+                  Java <span title="{{n.system.jvm.jre.vendor}}">{{n.system.jvm.jre.version}}</span><br/>
+                  Solr {{n.system.lucene['solr-impl-version'].split(" ")[0]}}<br>
+                </div>
+                <div class="node-spec" ng-click="toggleDetails(key)">
+                  <a ng-show="showDetails[key]">hide details...</a>
+                  <a ng-show="!showDetails[key]">show details...</a>
+                </div>
+              </td>
+              <td>
+                <div class="node-cpu">
+                  <span class="{{n.cpuPctStyle}}">{{n.cpuPct}}%</span>
+                </div>
+              </td>
+              <td>
+                <div class="node-heap" title="total: {{n.heapTotal}} free: {{n.heapFree}} used%: {{n.heapUsedPct}}%">
+                  <span class="{{n.heapUsedPctStyle}}">{{n.heapUsedPct}}%</span>
+                </div>
+                <div class="node-spec" ng-show="showDetails[key]">
+                  Max: {{n.heapTotal}}<br/>
+                  Used: {{n.heapUsed}}
+                </div>
+              </td>
+              <td class="scroll-height-250">
+                <div>
+                  <div class="node-disk" title="Available disk: {{n.diskTotal}} free: {{n.diskFree}} used by this node: {{n.size}}">
+                    {{n.size}}
+                  </div>
+                  <div class="node-spec" ng-show="showDetails[key]">
+                    Total #docs: {{n.numDocs}}<br/>
+                    Avg size/doc: {{n.sizePerDoc}}
+                  </div>
+                  <div id="chart{{n.id}}" ng-show="showDetails[key]"></div>
+                </div>
+              </td>
+              <td><div class="node-requests" title="1minRate: {{n.req1minRate}} 5minRate: {{n.req5minRate}} 15minRate: {{n.req15minRate}} p75: {{n.reqp75_ms}} p99: {{n.reqp99_ms}}">
+                RPM: {{n.req15minRate}}<br/>p95: {{n.reqp95_ms}}ms</div>
+              </td>
+              <td>
+                <div ng-show="!n.collections">(none)</div>
+                <div ng-repeat="c in n.collections | limitTo:showDetails[key]?999:2 track by $index">
+                  <a href="{{n.base_url + '/#/~collections/' + c}}">{{ c }}</a>
+                </div>
+                <div class="more" ng-show="n.collections.length > 2 && !showDetails[key]">
+                  <a ng-click="toggleDetails(key)">({{n.collections.length - 2}} more...)</a>
+                </div>
+              </td>
+              <td class="scroll-height-250">
+                  <div ng-show="!n.cores">(none)</div>
+                  <div ng-repeat="core in n.cores | limitTo:showDetails[key]?999:2 track by $index">
+                    <a class="{{core.leader ? 'leader' : 'replica'}}" href="{{core.base_url + '/#/' + core.core}}">{{ core.label }}</a> ({{core.numDocsHuman}} docs)
+                    <ul class="core-details" ng-show="showDetails[key]">
+                      <li>deleted: {{core.deletedDocsHuman}}</li>
+                      <li>warmupTime: {{core.warmupTime}}</li>
+                      <li ng-show="core.numDocs > 0">avg size/doc: {{core.avgSizePerDoc}}</li>
+                    </ul>
+                  </div>
+                  <div class="more" ng-show="n.cores.length > 2 && !showDetails[key]">
+                    <a ng-click="toggleDetails(key)">({{n.cores.length - 2}} more...)</a>
+                  </div>
+              </td>
+          </tr>
+        </tbody>
+      </table>
+    </div>
+    
     <div graph data="graphData" leaf-count="leafCount" helper-data="helperData" is-radial="isRadial" id="graph-content" class="content clearfix" ng-show="showGraph">
 
       <div id="canvas"></div>