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 2020/04/17 15:12:29 UTC

[lucene-solr] branch branch_8x updated: SOLR-14371 Zk StatusHandler should know about dynamic zk config (#1392)

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

janhoy pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 59cffaa  SOLR-14371 Zk StatusHandler should know about dynamic zk config (#1392)
59cffaa is described below

commit 59cffaa9c6a7702a6f3313266c88698dcbb899d2
Author: Jan Høydahl <ja...@users.noreply.github.com>
AuthorDate: Fri Apr 17 16:30:27 2020 +0200

    SOLR-14371 Zk StatusHandler should know about dynamic zk config (#1392)
    
    (cherry picked from commit 03363f413f2134594b012175deb3f10ec9384400)
    with java11 -> java8 modifications
---
 solr/CHANGES.txt                                   |   3 +
 .../solr/handler/admin/ZookeeperStatusHandler.java |  72 +++++++---
 .../solr/common/cloud/ZkDynamicConfigTest.java     |  62 +++++++++
 .../handler/admin/ZookeeperStatusHandlerTest.java  |  20 +--
 .../setting-up-an-external-zookeeper-ensemble.adoc |   2 +
 .../org/apache/solr/common/cloud/SolrZkClient.java |  25 ++++
 .../apache/solr/common/cloud/ZkDynamicConfig.java  | 146 +++++++++++++++++++++
 .../apache/solr/common/cloud/SolrZkClientTest.java |   8 +-
 solr/webapp/web/js/angular/controllers/cloud.js    |   5 +-
 solr/webapp/web/partials/cloud.html                |   3 +-
 10 files changed, 315 insertions(+), 31 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index e161c30..bbc3173 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -69,6 +69,9 @@ Bug Fixes
 
 * SOLR-14411: Fix Admin UI collection/core drop-downs placeholder text. Completes work started in SOLR-14359 (janhoy)
 
+* SOLR-14371: Zk StatusHandler now parse dynamic zk server config if supported, fixing Admin UI Zookeeper Status
+  screen in case dynamic reconfig host list differs from static zkHost string (janhoy)
+
 Other Changes
 ---------------------
 * SOLR-14197: SolrResourceLoader: marked many methods as deprecated, and in some cases rerouted exiting logic to avoid
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperStatusHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperStatusHandler.java
index 3b32bd0..80b3f74 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperStatusHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperStatusHandler.java
@@ -24,8 +24,8 @@ import java.io.PrintWriter;
 import java.io.Writer;
 import java.lang.invoke.MethodHandles;
 import java.net.Socket;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Locale;
@@ -33,6 +33,8 @@ import java.util.Map;
 import java.util.stream.Collectors;
 
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkDynamicConfig;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.RequestHandlerBase;
@@ -71,35 +73,67 @@ public class ZookeeperStatusHandler extends RequestHandlerBase {
     return Category.ADMIN;
   }
 
+  @SuppressWarnings("rawtypes")
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
     NamedList values = rsp.getValues();
     if (cores.isZooKeeperAware()) {
-      values.add("zkStatus", getZkStatus(cores.getZkController().getZkServerAddress()));
+      String zkHost = cores.getZkController().getZkServerAddress();
+      SolrZkClient zkClient = cores.getZkController().getZkClient();
+      final ZkDynamicConfig dynConfig = ZkDynamicConfig.parseLines(zkClient.getConfig());
+      values.add("zkStatus", getZkStatus(zkHost, dynConfig));
     } else {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The Zookeeper status API is only available in Cloud mode");
     }
   }
 
-  /*
-   Gets all info from ZK API and returns as a map
+  /**
+   * For each zk host, resolved either from zk connection string or from zk dynamic reconfiguration config,
+   * fetch all config and status info from ZK API and returns as a map, where key is hostname
+   * @param zkHost zookeeper connection string
+   * @param zkDynamicConfig list of zk dynamic config objects
+   * @return map of zookeeper config and status per zk host
    */
-  protected Map<String, Object> getZkStatus(String zkHost) {
-    Map<String, Object> zkStatus = new HashMap<>();
-    List<String> zookeepers = Arrays.asList(zkHost.split("/")[0].split(","));
-    List<Object> details = new ArrayList<>();
-    int numOk = 0;
+  protected Map<String, Object> getZkStatus(String zkHost, ZkDynamicConfig zkDynamicConfig) {
+    final ZkDynamicConfig hostsFromConnectionString = ZkDynamicConfig.fromZkConnectString(zkHost);
+    final ZkDynamicConfig zookeepers;
+    boolean dynamicReconfig;
+    final List<String> errors = new ArrayList<>();
     String status = STATUS_NA;
+
+    if (zkDynamicConfig.size() == 0) {
+      // Fallback to parsing zkHost for older zk servers without support for dynamic reconfiguration
+      dynamicReconfig = false;
+      zookeepers = hostsFromConnectionString;
+    } else {
+      dynamicReconfig = true;
+      List<String> connStringHosts = hostsFromConnectionString.getServers().stream()
+          .map(h -> h.resolveClientPortAddress() + ":" + h.clientPort)
+          .sorted().collect(Collectors.toList());
+      List<String> dynamicHosts = zkDynamicConfig.getServers().stream()
+          .map(h -> h.resolveClientPortAddress() + ":" + h.clientPort)
+          .sorted().collect(Collectors.toList());
+      if (!connStringHosts.containsAll(dynamicHosts)) {
+        errors.add("Your ZK connection string (" + connStringHosts.size() + " hosts) is different from the " +
+                "dynamic ensemble config (" + dynamicHosts.size() + " hosts). Solr does not currently support " +
+                "dynamic reconfiguration and will only be able to connect to the zk hosts in your connection string.");
+        status = STATUS_YELLOW;
+      }
+      zookeepers = zkDynamicConfig; // Clone input
+    }
+    final Map<String, Object> zkStatus = new HashMap<>();
+    final List<Object> details = new ArrayList<>();
+    int numOk = 0;
     int standalone = 0;
     int followers = 0;
     int reportedFollowers = 0;
     int leaders = 0;
-    List<String> errors = new ArrayList<>();
     zkStatus.put("ensembleSize", zookeepers.size());
     zkStatus.put("zkHost", zkHost);
-    for (String zk : zookeepers) {
+    for (ZkDynamicConfig.Server zk : zookeepers.getServers()) {
+      final String zkClientHostPort = zk.resolveClientPortAddress() + ":" + zk.clientPort;
       try {
-        Map<String, Object> stat = monitorZookeeper(zk);
+        Map<String, Object> stat = monitorZookeeper(zkClientHostPort);
         if (stat.containsKey("errors")) {
           errors.addAll((List<String>)stat.get("errors"));
           stat.remove("errors");
@@ -109,7 +143,7 @@ public class ZookeeperStatusHandler extends RequestHandlerBase {
           numOk++;
         }
         String state = String.valueOf(stat.get("zk_server_state"));
-        if ("follower".equals(state)) {
+        if ("follower".equals(state) || "observer".equals(state)) {
           followers++;
         } else if ("leader".equals(state)) {
           leaders++;
@@ -117,17 +151,21 @@ public class ZookeeperStatusHandler extends RequestHandlerBase {
         } else if ("standalone".equals(state)) {
           standalone++;
         }
+        if (zk.role != null) {
+          stat.put("role", zk.role);
+        }
       } catch (SolrException se) {
-        log.warn("Failed talking to zookeeper " + zk, se);
+        log.warn("Failed talking to zookeeper " + zkClientHostPort, se);
         errors.add(se.getMessage());
         Map<String, Object> stat = new HashMap<>();
-        stat.put("host", zk);
+        stat.put("host", zkClientHostPort);
         stat.put("ok", false);
         status = STATUS_YELLOW;
         details.add(stat);
       }
     }
     zkStatus.put("details", details);
+    zkStatus.put("dynamicReconfig", dynamicReconfig);
     if (followers+leaders > 0 && standalone > 0) {
       status = STATUS_RED;
       errors.add("The zk nodes do not agree on their mode, check details");
@@ -234,9 +272,9 @@ public class ZookeeperStatusHandler extends RequestHandlerBase {
 
     try (
         Socket socket = new Socket(host, port);
-        Writer writer = new OutputStreamWriter(socket.getOutputStream(), "utf-8");
+        Writer writer = new OutputStreamWriter(socket.getOutputStream(), StandardCharsets.UTF_8);
         PrintWriter out = new PrintWriter(writer, true);
-        BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream(), "utf-8"));) {
+        BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8))) {
       out.println(fourLetterWordCommand);
       List<String> response = in.lines().collect(Collectors.toList());
       log.debug("Got response from ZK on host {} and port {}: {}", host, port, response);
diff --git a/solr/core/src/test/org/apache/solr/common/cloud/ZkDynamicConfigTest.java b/solr/core/src/test/org/apache/solr/common/cloud/ZkDynamicConfigTest.java
new file mode 100644
index 0000000..87b4772
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/common/cloud/ZkDynamicConfigTest.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.common.cloud;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.junit.Test;
+
+public class ZkDynamicConfigTest extends SolrTestCaseJ4 {
+  @Test
+  public void parseLines() {
+    ZkDynamicConfig parsed = ZkDynamicConfig.parseLines(
+        "ignored-line\n" +
+            "server.1=zoo1:2780:2783:participant;0.0.0.0:2181\n" +
+            "server.2=zoo2:2781:2784:participant|zoo3:2783;2181\n" +
+            "server.3=zoo3:2782:2785;zoo3-client:2181\n" +
+            "version=400000003");
+    assertEquals(3, parsed.size());
+
+    assertEquals("zoo1", parsed.getServers().get(0).address);
+    assertEquals(Integer.valueOf(2780), parsed.getServers().get(0).leaderPort);
+    assertEquals(Integer.valueOf(2783), parsed.getServers().get(0).leaderElectionPort);
+    assertEquals("participant", parsed.getServers().get(0).role);
+    assertEquals("0.0.0.0", parsed.getServers().get(0).clientPortAddress);
+    assertEquals(Integer.valueOf(2181), parsed.getServers().get(0).clientPort);
+    assertEquals("zoo1", parsed.getServers().get(0).resolveClientPortAddress());
+
+    // |<host2> is ignored
+    assertEquals("participant", parsed.getServers().get(1).role);
+    assertNull(parsed.getServers().get(1).clientPortAddress);
+    assertEquals("zoo2", parsed.getServers().get(1).resolveClientPortAddress());
+    assertEquals(Integer.valueOf(2181), parsed.getServers().get(1).clientPort);
+
+    // role optional
+    assertNull(parsed.getServers().get(2).role);
+    assertEquals("zoo3-client", parsed.getServers().get(2).clientPortAddress);
+    assertEquals("zoo3-client", parsed.getServers().get(2).resolveClientPortAddress());
+  }
+
+  @Test(expected = SolrException.class)
+  public void parseLinesInvalid() {
+    ZkDynamicConfig.parseLines(
+        "server.1=zoo2:2781:2784:participant|zoo3:2783;0.0.0.0:2181\n" +
+            "server.2=zoo3:2782\n" + // This line fails as it lacks mandatory parts
+            "version=400000003");
+  }
+}
\ No newline at end of file
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperStatusHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperStatusHandlerTest.java
index f795471..693accf 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperStatusHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperStatusHandlerTest.java
@@ -18,7 +18,6 @@
 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.Collections;
@@ -35,6 +34,7 @@ import org.apache.solr.client.solrj.request.GenericSolrRequest;
 import org.apache.solr.client.solrj.response.DelegationTokenResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkDynamicConfig;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.junit.After;
@@ -44,8 +44,6 @@ import org.junit.Test;
 import org.mockito.Answers;
 import org.mockito.ArgumentMatchers;
 import org.noggit.JSONUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyString;
@@ -53,8 +51,6 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 public class ZookeeperStatusHandlerTest extends SolrCloudTestCase {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(1)
@@ -121,15 +117,22 @@ public class ZookeeperStatusHandlerTest extends SolrCloudTestCase {
     when(zkStatusHandler.getZkRawResponse("zoo3:2181", "conf")).thenReturn(
         Arrays.asList("clientPort=2181"));
 
-    when(zkStatusHandler.getZkStatus(anyString())).thenCallRealMethod();
+    when(zkStatusHandler.getZkStatus(anyString(), any())).thenCallRealMethod();
     when(zkStatusHandler.monitorZookeeper(anyString())).thenCallRealMethod();
     when(zkStatusHandler.validateZkRawResponse(ArgumentMatchers.any(), any(), any())).thenAnswer(Answers.CALLS_REAL_METHODS);
 
-    Map<String, Object> mockStatus = zkStatusHandler.getZkStatus("zoo1:2181,zoo2:2181,zoo3:2181");
+    ZkDynamicConfig zkDynamicConfig = ZkDynamicConfig.parseLines(
+        "server.1=zoo1:2780:2783:participant;0.0.0.0:2181\n" +
+            "server.2=zoo2:2781:2784:participant;0.0.0.0:2181\n" +
+            "server.3=zoo3:2782:2785:participant;0.0.0.0:2181\n" +
+            "version=400000003");
+    Map<String, Object> mockStatus = zkStatusHandler.getZkStatus("zoo4:2181,zoo5:2181,zoo6:2181", zkDynamicConfig);
     String expected = "{\n" +
+        "  \"dynamicReconfig\":true,\n" +
         "  \"ensembleSize\":3,\n" +
         "  \"details\":[\n" +
         "    {\n" +
+        "      \"role\":\"participant\",\n" +
         "      \"zk_version\":\"3.5.5-390fe37ea45dee01bf87dc1c042b5e3dcce88653, built on 05/03/2019 12:07 GMT\",\n" +
         "      \"zk_avg_latency\":\"1\",\n" +
         "      \"host\":\"zoo1:2181\",\n" +
@@ -142,8 +145,9 @@ public class ZookeeperStatusHandlerTest extends SolrCloudTestCase {
         "    {\n" +
         "      \"host\":\"zoo3:2181\",\n" +
         "      \"ok\":false}],\n" +
-        "  \"zkHost\":\"zoo1:2181,zoo2:2181,zoo3:2181\",\n" +
+        "  \"zkHost\":\"zoo4:2181,zoo5:2181,zoo6:2181\",\n" +
         "  \"errors\":[\n" +
+        "    \"Your ZK connection string (3 hosts) is different from the dynamic ensemble config (3 hosts). Solr does not currently support dynamic reconfiguration and will only be able to connect to the zk hosts in your connection string.\",\n" +
         "    \"Unexpected line in 'conf' response from Zookeeper zoo1:2181: thisIsUnexpected\",\n" +
         "    \"Empty response from Zookeeper zoo2:2181\",\n" +
         "    \"Could not execute mntr towards ZK host zoo3:2181. Add this line to the 'zoo.cfg' configuration file on each zookeeper node: '4lw.commands.whitelist=mntr,conf,ruok'. See also chapter 'Setting Up an External ZooKeeper Ensemble' in the Solr Reference Guide.\"],\n" +
diff --git a/solr/solr-ref-guide/src/setting-up-an-external-zookeeper-ensemble.adoc b/solr/solr-ref-guide/src/setting-up-an-external-zookeeper-ensemble.adoc
index 837c4d9..48a7345 100644
--- a/solr/solr-ref-guide/src/setting-up-an-external-zookeeper-ensemble.adoc
+++ b/solr/solr-ref-guide/src/setting-up-an-external-zookeeper-ensemble.adoc
@@ -269,6 +269,8 @@ When starting Solr, you must provide an address for ZooKeeper or Solr won't know
 
 When referring to the location of ZooKeeper within Solr, it's best to use the addresses of all the servers in the ensemble. If one happens to be down, Solr will automatically be able to send its request to another server in the list.
 
+ZooKeeper version 3.5 and later supports dynamic reconfiguration of server addresses and roles. But note that Solr will only be able to talk to the servers listed in the static ZooKeeper connect string.
+
 === Using a chroot
 
 If your ensemble is or will be shared among other systems besides Solr, you should consider defining application-specific _znodes_, or a hierarchical namespace that will only include Solr's files.
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
index 2816ba3..cc0117e 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
@@ -56,6 +56,7 @@ import org.apache.zookeeper.Op;
 import org.apache.zookeeper.OpResult;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
@@ -751,6 +752,30 @@ public class SolrZkClient implements Closeable {
     return zkServerAddress;
   }
 
+  /**
+   * Gets the raw config node /zookeeper/config as returned by server. Response may look like
+   * <pre>
+   * server.1=localhost:2780:2783:participant;localhost:2791
+   * server.2=localhost:2781:2784:participant;localhost:2792
+   * server.3=localhost:2782:2785:participant;localhost:2793
+   * version=400000003
+   * </pre>
+   * @return Multi line string representing the config. For standalone ZK this will return empty string
+   */
+  public String getConfig() {
+    try {
+      Stat stat = new Stat();
+      keeper.sync(ZooDefs.CONFIG_NODE, null, null);
+      byte[] data = keeper.getConfig(false, stat);
+      if (data == null || data.length == 0) {
+        return "";
+      }
+      return new String(data, StandardCharsets.UTF_8);
+    } catch (KeeperException|InterruptedException ex) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Failed to get config from zookeeper", ex);
+    }
+  }
+
   public ZkACLProvider getZkACLProvider() {
     return zkACLProvider;
   }
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkDynamicConfig.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkDynamicConfig.java
new file mode 100644
index 0000000..ad265a3
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkDynamicConfig.java
@@ -0,0 +1,146 @@
+/*
+ * 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.common.cloud;
+
+import java.io.BufferedReader;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.StringUtils;
+
+/**
+ * Class holding the dynamic config of a Zookeeper ensemble as fetched from znode <code>/zookeeper/config</code>.
+ */
+public class ZkDynamicConfig {
+  // server.<positive id> = <address1>:<port1>:<port2>[:role][|<address2>:<port2>...];[<client port address>:]<client port>
+  // TODO: Add support for handling multiple address specs per server line, how we simply ignore all but the first
+  public static final Pattern linePattern = Pattern.compile("server\\.(?<serverId>\\d+) ?= ?(?<address>[^:]+):(?<leaderPort>\\d+):(?<leaderElectionPort>\\d+)(:(?<role>.*?))?(\\|.*?)?(;((?<clientPortAddress>.*?):)?(?<clientPort>\\d+))?");
+
+  private List<Server> servers = new ArrayList<>();
+  private String version = "";
+
+  private ZkDynamicConfig() { /* Use static factory methods */ }
+
+  /**
+   * Parse a raw multi line config string with the full content of znode /zookeeper/config.
+   * @param lines the multi line config string. If empty or null, this will return an empty list
+   * @return an instance of ZkDynamicConfig
+   */
+  public static ZkDynamicConfig parseLines(String lines) {
+    ZkDynamicConfig zkDynamicConfig = new ZkDynamicConfig();
+    if (!StringUtils.isEmpty(lines)) {
+      new BufferedReader(new StringReader(lines)).lines().forEach(l -> {
+        if (l.startsWith("version=")) {
+          zkDynamicConfig.version = l.split("=")[1];
+        }
+        if (l.startsWith("server.")) {
+          zkDynamicConfig.servers.add(Server.parseLine(l));
+        }
+      });
+    }
+    return zkDynamicConfig;
+  }
+
+  /**
+   * Creates an instance based on a zookeeper connect string on format <code>host:port,host:port[/chroot]</code>
+   * @param zkHost zk connect string
+   * @return instance of ZkDynamicConfig
+   */
+  public static ZkDynamicConfig fromZkConnectString(String zkHost) {
+    ZkDynamicConfig zkDynamicConfig = new ZkDynamicConfig();
+    zkDynamicConfig.servers = Arrays.stream(zkHost.split("/")[0].split(","))
+        .map(h -> new ZkDynamicConfig.Server(
+            null,
+            null,
+            null,
+            null,
+            null,
+            h.split(":")[0],
+            h.contains(":") ? Integer.parseInt(h.split(":")[1]) : 2181)
+        ).collect(Collectors.toList());
+    return zkDynamicConfig;
+  }
+
+  public List<Server> getServers() {
+    return servers;
+  }
+
+  public String getVersion() {
+    return version;
+  }
+
+  public int size() {
+    return servers.size();
+  }
+
+  /**
+   * Object representing one line in Zk dynamic config
+   */
+  public static class Server {
+    public final Integer serverId;
+    public final String address;
+    public final Integer leaderPort;
+    public final Integer leaderElectionPort;
+    public final String role;
+    public final String clientPortAddress;
+    public final Integer clientPort;
+
+    Server(Integer serverId, String address, Integer leaderPort, Integer leaderElectionPort, String role, String clientPortAddress, Integer clientPort) {
+      this.serverId = serverId;
+      this.address = address;
+      this.leaderPort = leaderPort;
+      this.leaderElectionPort = leaderElectionPort;
+      this.role = role;
+      this.clientPortAddress = clientPortAddress;
+      this.clientPort = clientPort;
+    }
+
+    /**
+     * Resolve the most likely address, first trying 'clientPortAddress', falling back to 'address'
+     * @return a string with client address, without port
+     */
+    public String resolveClientPortAddress() {
+      return ("0.0.0.0".equals(clientPortAddress) || clientPortAddress == null ? address : clientPortAddress);
+    }
+
+    /**
+     * Parse a single zk config server line
+     */
+    public static Server parseLine(String line) {
+      Matcher m = linePattern.matcher(line);
+      if (!m.matches()) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Could not parse dynamic zk config line: " + line);
+      }
+      return new Server(
+          Integer.parseInt(m.group("serverId")),
+          m.group("address"),
+          Integer.parseInt(m.group("leaderPort")),
+          Integer.parseInt(m.group("leaderElectionPort")),
+          m.group("role"),
+          m.group("clientPortAddress"),
+          Integer.parseInt(m.group("clientPort"))
+      );
+    }
+  }
+}
diff --git a/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java b/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java
index cf9f694..561f0e4 100644
--- a/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java
@@ -208,6 +208,12 @@ public class SolrZkClientTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void getConfig() {
+    // As the embedded ZK is hardcoded to standalone, there is no way to test actual config data here
+    assertEquals("", defaultClient.getConfig());
+  }
+
+  @Test
   public void testCheckInterrupted() {
     assertFalse(Thread.currentThread().isInterrupted());
     SolrZkClient.checkInterrupted(new RuntimeException());
@@ -215,6 +221,4 @@ public class SolrZkClientTest extends SolrCloudTestCase {
     SolrZkClient.checkInterrupted(new InterruptedException());
     assertTrue(Thread.currentThread().isInterrupted());
   }
-
-
 }
diff --git a/solr/webapp/web/js/angular/controllers/cloud.js b/solr/webapp/web/js/angular/controllers/cloud.js
index 57b58aa..27be1ad 100644
--- a/solr/webapp/web/js/angular/controllers/cloud.js
+++ b/solr/webapp/web/js/angular/controllers/cloud.js
@@ -536,10 +536,9 @@ var zkStatusSubController = function($scope, ZookeeperStatus) {
           "zk_avg_latency", "zk_max_file_descriptor_count", "zk_watch_count", 
           "zk_packets_sent", "zk_packets_received",
           "tickTime", "maxClientCnxns", "minSessionTimeout", "maxSessionTimeout"];
-        $scope.ensembleMainKeys = ["serverId", "electionPort", "quorumPort"];
+        $scope.ensembleMainKeys = ["serverId", "electionPort", "quorumPort", "role"];
         $scope.ensembleDetailKeys = ["peerType", "electionAlg", "initLimit", "syncLimit",
-          "zk_followers", "zk_synced_followers", "zk_pending_syncs",
-          "server.1", "server.2", "server.3", "server.4", "server.5"];
+          "zk_followers", "zk_synced_followers", "zk_pending_syncs"];
         $scope.notEmptyRow = function(key) {
           for (hostId in $scope.zkState.details) {
             if (key in $scope.zkState.details[hostId]) return true;
diff --git a/solr/webapp/web/partials/cloud.html b/solr/webapp/web/partials/cloud.html
index c2a878a..01d6349 100644
--- a/solr/webapp/web/partials/cloud.html
+++ b/solr/webapp/web/partials/cloud.html
@@ -36,6 +36,7 @@ limitations under the License.
       <div>ZK connection string: {{zkState.zkHost}}</div>
       <div>Ensemble size: {{zkState.ensembleSize}}</div>
       <div>Ensemble mode: {{zkState.mode}}</div>
+      <div>Dynamic reconfig enabled: {{zkState.dynamicReconfig}}</div>
       
       <table id="zk-table">
         <thead>
@@ -61,7 +62,7 @@ limitations under the License.
             <td ng-class="details"></td>
             <td ng-repeat="host in zkState.details" ng-class="details"></td>
           </tr>
-          <tr ng-repeat="key in detailKeys" ng-show="showDetails">
+          <tr ng-repeat="key in detailKeys | filter: notEmptyRow" ng-show="showDetails">
             <td>{{key}}</td>
             <td ng-repeat="host in zkState.details">
               {{host[key]}}