You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2020/04/27 10:39:35 UTC

[lucene-solr] branch branch_8x updated: SOLR-13942: A read API at /api/cluster/zk/* to fetch raw ZK data and view contents of a ZK direcory

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

noble 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 f5c0f5c  SOLR-13942: A read API at /api/cluster/zk/* to fetch raw ZK data and view contents of a ZK direcory
f5c0f5c is described below

commit f5c0f5c3e4a14ac0880c37a208912947cb6f14fb
Author: noble <no...@apache.org>
AuthorDate: Mon Apr 27 20:39:13 2020 +1000

    SOLR-13942: A read API at /api/cluster/zk/* to fetch raw ZK data and view contents of a ZK direcory
---
 solr/CHANGES.txt                                   |   3 +
 .../src/java/org/apache/solr/api/AnnotatedApi.java |  89 ++++++++-----
 solr/core/src/java/org/apache/solr/api/ApiBag.java |  13 ++
 .../src/java/org/apache/solr/api/EndPoint.java     |   2 +-
 .../java/org/apache/solr/core/CoreContainer.java   |  12 +-
 .../solr/handler/admin/ZookeeperReadAPI.java       | 145 +++++++++++++++++++++
 .../solr/security/PermissionNameProvider.java      |   1 +
 .../solr/handler/admin/TestApiFramework.java       |  27 +++-
 .../solr/handler/admin/ZookeeperReadAPITest.java   | 101 ++++++++++++++
 9 files changed, 347 insertions(+), 46 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4c8a7d5..d1681e0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -13,6 +13,9 @@ New Features
 * SOLR-14210: HealthCheckHandler can now require that all cores are healthy before returning 'OK'
   To enable, add &requireHealthyCores=true to enable (janhoy, Houston Putman, shalin)
 
+* SOLR-13942: A read API at /api/cluster/zk/* to fetch raw ZK data and view contents of a ZK direcory (noble)
+
+
 Improvements
 ---------------------
 * SOLR-14316: Remove unchecked type conversion warning in JavaBinCodec's readMapEntry's equals() method
diff --git a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
index ae04d03..a37240a 100644
--- a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
+++ b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
@@ -63,33 +63,60 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
 
   public static final String ERR = "Error executing commands :";
   private EndPoint endPoint;
-  private Map<String, Cmd> commands = new HashMap<>();
+  private final Map<String, Cmd> commands ;
+  private final Cmd singletonCommand;
   private final Api fallback;
 
-  public AnnotatedApi(Object obj) {
-    this(obj, null);
-  }
-
-  public AnnotatedApi(Object obj, Api fallback) {
-    super(readSpec(obj.getClass()));
-    this.fallback = fallback;
-    Class<?> klas = obj.getClass();
+  public static List<Api> getApis(Object obj) {
+    Class<? extends Object> klas = obj.getClass();
     if (!Modifier.isPublic(klas.getModifiers())) {
       throw new RuntimeException(obj.getClass().getName() + " is not public");
     }
-
-    endPoint = klas.getAnnotation(EndPoint.class);
-
-    for (Method m : klas.getDeclaredMethods()) {
-      Command command = m.getAnnotation(Command.class);
-      if (command == null) continue;
-
-      if (commands.containsKey(command.name())) {
-        throw new RuntimeException("Duplicate commands " + command.name());
+    if (klas.getAnnotation(EndPoint.class) != null) {
+      EndPoint endPoint = klas.getAnnotation(EndPoint.class);
+      List<Method> methods = new ArrayList<>();
+      Map<String, Cmd> commands = new HashMap<>();
+      for (Method m : klas.getDeclaredMethods()) {
+        Command command = m.getAnnotation(Command.class);
+        if (command != null) {
+          methods.add(m);
+          if (commands.containsKey(command.name())) {
+            throw new RuntimeException("Duplicate commands " + command.name());
+          }
+          commands.put(command.name(), new Cmd(command.name(), obj, m));
+        }
       }
-      commands.put(command.name(), new Cmd(command, obj, m));
+      if (commands.isEmpty()) {
+        throw new RuntimeException("No method with @Command in class: " + obj.getClass().getName());
+      }
+      SpecProvider specProvider = readSpec(endPoint, methods);
+      return Collections.singletonList(new AnnotatedApi(specProvider, endPoint, commands, null));
+    } else {
+      List<Api> apis = new ArrayList<>();
+      for (Method m : klas.getDeclaredMethods()) {
+        EndPoint endPoint = m.getAnnotation(EndPoint.class);
+        if (endPoint == null) continue;
+        if (!Modifier.isPublic(m.getModifiers())) {
+          throw new RuntimeException("Non public method " + m.toGenericString());
+        }
+        Cmd cmd = new Cmd("", obj, m);
+        SpecProvider specProvider = readSpec(endPoint, Collections.singletonList(m));
+        apis.add(new AnnotatedApi(specProvider, endPoint, Collections.singletonMap("", cmd), null));
+      }
+      if (apis.isEmpty()) {
+        throw new RuntimeException("Invalid Class : " + obj.getClass().getName() + " No @EndPoints");
+      }
+      return apis;
     }
+  }
 
+
+  private AnnotatedApi(SpecProvider specProvider, EndPoint endPoint, Map<String, Cmd> commands, Api fallback) {
+    super(specProvider);
+    this.endPoint = endPoint;
+    this.fallback = fallback;
+    this.commands = commands;
+    this.singletonCommand = commands.get("");
   }
 
   @Override
@@ -97,10 +124,7 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
     return endPoint.permission();
   }
 
-  private static SpecProvider readSpec(Class klas) {
-    EndPoint endPoint = (EndPoint) klas.getAnnotation(EndPoint.class);
-    if (endPoint == null)
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Invalid class :  " + klas.getName());
+  private static SpecProvider readSpec(EndPoint endPoint, List<Method> m) {
     return () -> {
       Map map = new LinkedHashMap();
       List<String> methods = new ArrayList<>();
@@ -111,7 +135,7 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
       map.put("url", new ValidatingJsonMap(Collections.singletonMap("paths", Arrays.asList(endPoint.path()))));
       Map<String, Object> cmds = new HashMap<>();
 
-      for (Method method : klas.getMethods()) {
+      for (Method method : m) {
         Command command = method.getAnnotation(Command.class);
         if (command != null && !command.name().isEmpty()) {
           cmds.put(command.name(), AnnotatedApi.createSchema(method));
@@ -123,18 +147,14 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
       return new ValidatingJsonMap(map);
     };
 
-
   }
 
 
   @Override
   public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
-    if (commands.size() == 1) {
-      Cmd cmd = commands.get("");
-      if (cmd != null) {
-        cmd.invoke(req, rsp, null);
-        return;
-      }
+    if (singletonCommand != null) {
+      singletonCommand.invoke(req, rsp, null);
+      return;
     }
 
     List<CommandOperation> cmds = req.getCommands(true);
@@ -167,8 +187,8 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
 
   }
 
-  class Cmd {
-    final Command command;
+  static class Cmd {
+    final String command;
     final Method method;
     final Object obj;
     ObjectMapper mapper = SolrJacksonAnnotationInspector.createObjectMapper();
@@ -177,7 +197,7 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
     boolean isWrappedInPayloadObj = false;
 
 
-    Cmd(Command command, Object obj, Method method) {
+    Cmd(String command, Object obj, Method method) {
       if (Modifier.isPublic(method.getModifiers())) {
         this.command = command;
         this.obj = obj;
@@ -204,7 +224,6 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
           } else {
             c = (Class) t;
           }
-
         }
         if (parameterTypes.length > 3) {
           throw new RuntimeException("Invalid params count for method " + method);
diff --git a/solr/core/src/java/org/apache/solr/api/ApiBag.java b/solr/core/src/java/org/apache/solr/api/ApiBag.java
index 36f012c..84facfa 100644
--- a/solr/core/src/java/org/apache/solr/api/ApiBag.java
+++ b/solr/core/src/java/org/apache/solr/api/ApiBag.java
@@ -66,6 +66,19 @@ public class ApiBag {
     this.isCoreSpecific = isCoreSpecific;
   }
 
+  /**Register a POJO annotated with {@link EndPoint}
+   * @param o the instance to be used for invocations
+   */
+  public synchronized List<Api> registerObject(Object o) {
+    List<Api> l = AnnotatedApi.getApis(o);
+    for (Api api : l) {
+      register(api, Collections.EMPTY_MAP);
+    }
+    return l;
+  }
+  public synchronized void register(Api api) {
+    register(api, Collections.EMPTY_MAP);
+  }
   public synchronized void register(Api api, Map<String, String> nameSubstitutes) {
     try {
       validateAndRegister(api, nameSubstitutes);
diff --git a/solr/core/src/java/org/apache/solr/api/EndPoint.java b/solr/core/src/java/org/apache/solr/api/EndPoint.java
index 6cbe500..f0358dc 100644
--- a/solr/core/src/java/org/apache/solr/api/EndPoint.java
+++ b/solr/core/src/java/org/apache/solr/api/EndPoint.java
@@ -26,7 +26,7 @@ import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.security.PermissionNameProvider;
 
 @Retention(RetentionPolicy.RUNTIME)
-@Target({ElementType.TYPE})
+@Target({ElementType.TYPE, ElementType.METHOD})
 public @interface EndPoint {
   SolrRequest.METHOD[] method();
 
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 cddf131..4ea1cd8 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -51,7 +51,6 @@ import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.store.Directory;
-import org.apache.solr.api.AnnotatedApi;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
@@ -97,6 +96,7 @@ import org.apache.solr.handler.admin.SecurityConfHandler;
 import org.apache.solr.handler.admin.SecurityConfHandlerLocal;
 import org.apache.solr.handler.admin.SecurityConfHandlerZk;
 import org.apache.solr.handler.admin.ZookeeperInfoHandler;
+import org.apache.solr.handler.admin.ZookeeperReadAPI;
 import org.apache.solr.handler.admin.ZookeeperStatusHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.logging.LogWatcher;
@@ -611,8 +611,8 @@ public class CoreContainer {
     }
 
     packageStoreAPI = new PackageStoreAPI(this);
-    containerHandlers.getApiBag().register(new AnnotatedApi(packageStoreAPI.readAPI), Collections.EMPTY_MAP);
-    containerHandlers.getApiBag().register(new AnnotatedApi(packageStoreAPI.writeAPI), Collections.EMPTY_MAP);
+    containerHandlers.getApiBag().registerObject(packageStoreAPI.readAPI);
+    containerHandlers.getApiBag().registerObject(packageStoreAPI.writeAPI);
 
     metricManager = new SolrMetricManager(loader, cfg.getMetricsConfig());
     String registryName = SolrMetricManager.getRegistryName(SolrInfoBean.Group.node);
@@ -648,8 +648,10 @@ public class CoreContainer {
           solrMetricsContext.metricManager, solrMetricsContext.registry, solrMetricsContext.tag, "/authentication/pki");
       TracerConfigurator.loadTracer(loader, cfg.getTracerConfiguratorPluginInfo(), getZkController().getZkStateReader());
       packageLoader = new PackageLoader(this);
-      containerHandlers.getApiBag().register(new AnnotatedApi(packageLoader.getPackageAPI().editAPI), Collections.EMPTY_MAP);
-      containerHandlers.getApiBag().register(new AnnotatedApi(packageLoader.getPackageAPI().readAPI), Collections.EMPTY_MAP);
+      containerHandlers.getApiBag().registerObject(packageLoader.getPackageAPI().editAPI);
+      containerHandlers.getApiBag().registerObject(packageLoader.getPackageAPI().readAPI);
+      ZookeeperReadAPI zookeeperReadAPI = new ZookeeperReadAPI(this);
+      containerHandlers.getApiBag().registerObject(zookeeperReadAPI);
     }
 
     MDCLoggingContext.setNode(this);
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperReadAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperReadAPI.java
new file mode 100644
index 0000000..60a53b8
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperReadAPI.java
@@ -0,0 +1,145 @@
+/*
+ * 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.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.api.EndPoint;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.impl.BinaryResponseParser;
+import org.apache.solr.client.solrj.impl.XMLResponseParser;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.RequestHandlerUtils;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+
+import static org.apache.solr.common.params.CommonParams.OMIT_HEADER;
+import static org.apache.solr.common.params.CommonParams.WT;
+import static org.apache.solr.response.RawResponseWriter.CONTENT;
+import static org.apache.solr.security.PermissionNameProvider.Name.COLL_READ_PERM;
+import static org.apache.solr.security.PermissionNameProvider.Name.ZK_READ_PERM;
+
+/**
+ * Exposes the content of the Zookeeper
+ * This is an expert feature that exposes the data inside the back end zookeeper.This API may change or
+ * be removed in future versions.
+ * This is not a public API. The data that is returned is not guaranteed to remain same
+ * across releases, as the data stored in Zookeeper may change from time to time.
+ * @lucene.experimental
+ */
+
+public class ZookeeperReadAPI {
+  private final CoreContainer coreContainer;
+
+  public ZookeeperReadAPI(CoreContainer coreContainer) {
+    this.coreContainer = coreContainer;
+  }
+  @EndPoint(path = "/cluster/zk/data/*",
+      method = SolrRequest.METHOD.GET,
+      permission = COLL_READ_PERM)
+  public void readNode(SolrQueryRequest req, SolrQueryResponse rsp) {
+    String path = req.getPathTemplateValues().get("*");
+    if (path == null || path.isEmpty()) path = "/";
+    byte[] d = null;
+    try {
+      d = coreContainer.getZkController().getZkClient().getData(path, null, null, false);
+    } catch (KeeperException.NoNodeException e) {
+      throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "No such node: " + path);
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unexpected error", e);
+    }
+    if (d == null || d.length == 0) {
+      rsp.add(path, null);
+      return;
+    }
+
+    Map<String, String> map = new HashMap<>(1);
+    map.put(WT, "raw");
+    map.put(OMIT_HEADER, "true");
+    req.setParams(SolrParams.wrapDefaults(new MapSolrParams(map), req.getParams()));
+
+    String mime = BinaryResponseParser.BINARY_CONTENT_TYPE;
+
+    if (d[0] == '{') mime = CommonParams.JSON_MIME;
+    if (d[0] == '<' || d[1] == '?') mime = XMLResponseParser.XML_CONTENT_TYPE;
+    rsp.add(CONTENT, new ContentStreamBase.ByteArrayStream(d, null, mime));
+  }
+
+  @EndPoint(path = "/cluster/zk/ls/*",
+      method = SolrRequest.METHOD.GET,
+      permission = ZK_READ_PERM)
+  public void listNodes(SolrQueryRequest req, SolrQueryResponse rsp) {
+    String path = req.getPathTemplateValues().get("*");
+    if (path == null || path.isEmpty()) path = "/";
+    try {
+      List<String> l = coreContainer.getZkController().getZkClient().getChildren(path, null, false);
+      String prefix = path.endsWith("/") ? path : path + "/";
+
+      Map<String , Stat> stats = new LinkedHashMap<>();
+      for (String s : l) {
+        try {
+          stats.put(s, coreContainer.getZkController().getZkClient().exists(prefix + s, null, false));
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      }
+      rsp.add(path, (MapWriter) ew -> {
+        for (Map.Entry<String, Stat> e : stats.entrySet()) {
+          printStat(ew, e.getKey(), e.getValue());
+        }
+      });
+    } catch (KeeperException.NoNodeException e) {
+      throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "No such node :"+ path);
+    } catch (Exception e) {
+      rsp.add(CONTENT, new ContentStreamBase.StringStream(Utils.toJSONString(Collections.singletonMap("error", e.getMessage()))));
+    } finally {
+      RequestHandlerUtils.addExperimentalFormatWarning(rsp);
+    }
+  }
+
+  private void printStat(MapWriter.EntryWriter ew, String s, Stat stat) throws IOException {
+    ew.put(s, (MapWriter) ew1 -> {
+      ew1.put("version", stat.getVersion());
+      ew1.put("aversion", stat.getAversion());
+      ew1.put("children", stat.getNumChildren());
+      ew1.put("ctime", stat.getCtime());
+      ew1.put("cversion", stat.getCversion());
+      ew1.put("czxid", stat.getCzxid());
+      ew1.put("ephemeralOwner", stat.getEphemeralOwner());
+      ew1.put("mtime", stat.getMtime());
+      ew1.put("mzxid", stat.getMzxid());
+      ew1.put("pzxid", stat.getPzxid());
+      ew1.put("dataLength", stat.getDataLength());
+    });
+  }
+
+}
diff --git a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
index b5d4099..ef6ae26 100644
--- a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
@@ -38,6 +38,7 @@ public interface PermissionNameProvider {
     COLL_READ_PERM("collection-admin-read", null),
     CORE_READ_PERM("core-admin-read", null),
     CORE_EDIT_PERM("core-admin-edit", null),
+    ZK_READ_PERM("zk-read", null),
     READ_PERM("read", "*"),
     UPDATE_PERM("update", "*"),
     CONFIG_EDIT_PERM("config-edit", unmodifiableSet(new HashSet<>(asList("*", null)))),
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
index f979831..32d1f58 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
@@ -28,7 +28,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.api.AnnotatedApi;
 import org.apache.solr.api.Api;
 import org.apache.solr.api.ApiBag;
 import org.apache.solr.api.Command;
@@ -173,10 +172,9 @@ public class TestApiFramework extends SolrTestCaseJ4 {
     Utils.fromJSONString(json);
 
     ApiBag apiBag = new ApiBag(false);
-    AnnotatedApi api = new AnnotatedApi(new ApiTest());
-    apiBag.register(api, Collections.emptyMap());
+    List<Api> apis =  apiBag.registerObject(new ApiTest());
 
-    ValidatingJsonMap spec = api.getSpec();
+    ValidatingJsonMap spec = apis.get(0).getSpec();
 
     assertEquals("POST", spec._getStr("/methods[0]",null) );
     assertEquals("POST", spec._getStr("/methods[0]",null) );
@@ -226,10 +224,17 @@ public class TestApiFramework extends SolrTestCaseJ4 {
 
   public void testAnnotatedApi() {
     ApiBag apiBag = new ApiBag(false);
-    apiBag.register(new AnnotatedApi(new DummyTest()), Collections.emptyMap());
+    apiBag.registerObject(new DummyTest());
     SolrQueryResponse rsp = v2ApiInvoke(apiBag, "/node/filestore/package/mypkg/jar1.jar", "GET",
         new ModifiableSolrParams(), null);
     assertEquals("/package/mypkg/jar1.jar", rsp.getValues().get("path"));
+
+    apiBag = new ApiBag(false);
+    apiBag.registerObject(new DummyTest1());
+    rsp = v2ApiInvoke(apiBag, "/node/filestore/package/mypkg/jar1.jar", "GET",
+        new ModifiableSolrParams(), null);
+    assertEquals("/package/mypkg/jar1.jar", rsp.getValues().get("path"));
+
   }
 
   @EndPoint(
@@ -244,6 +249,18 @@ public class TestApiFramework extends SolrTestCaseJ4 {
     }
   }
 
+
+  public class DummyTest1 {
+    @EndPoint(
+        path = "/node/filestore/*",
+        method = SolrRequest.METHOD.GET,
+        permission = PermissionNameProvider.Name.ALL)
+    public void read(SolrQueryRequest req, SolrQueryResponse rsp) {
+      rsp.add("FSRead.called", "true");
+      rsp.add("path", req.getPathTemplateValues().get("*"));
+    }
+  }
+
   private static SolrQueryResponse v2ApiInvoke(ApiBag bag, String uri, String method, SolrParams params, InputStream payload) {
     if (params == null) params = new ModifiableSolrParams();
     SolrQueryResponse rsp = new SolrQueryResponse();
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperReadAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperReadAPITest.java
new file mode 100644
index 0000000..d7ad7c6
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperReadAPITest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.util.Utils;
+import org.apache.zookeeper.CreateMode;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.util.StrUtils.split;
+import static org.apache.solr.common.util.Utils.getObjectByPath;
+
+public class ZookeeperReadAPITest extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  @After
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+  }
+
+  @Test
+  public void testZkread() throws Exception {
+    URL baseUrl = cluster.getJettySolrRunner(0).getBaseUrl();
+    String basezk = baseUrl.toString().replace("/solr", "/api") + "/cluster/zk/data";
+    String basezkls = baseUrl.toString().replace("/solr", "/api") + "/cluster/zk/ls";
+
+    try (HttpSolrClient client = new HttpSolrClient.Builder(baseUrl.toString()).build()) {
+      Object o = Utils.executeGET(client.getHttpClient(),
+          basezk + "/security.json",
+          Utils.JSONCONSUMER);
+      assertNotNull(o);
+      o = Utils.executeGET(client.getHttpClient(),
+          basezkls + "/configs",
+          Utils.JSONCONSUMER);
+      assertEquals("0", String.valueOf(getObjectByPath(o, true, split(":/configs:_default:dataLength", ':'))));
+      assertEquals("0", String.valueOf(getObjectByPath(o, true, split(":/configs:conf:dataLength", ':'))));
+
+      o = Utils.executeGET(client.getHttpClient(),
+          basezk + "/configs",
+          Utils.JSONCONSUMER);
+      assertTrue(((Map)o).containsKey("/configs"));
+      assertNull(((Map)o).get("/configs"));
+
+      byte[] bytes = new byte[1024 * 5];
+      for (int i = 0; i < bytes.length; i++) {
+        bytes[i] = (byte) random().nextInt(128);
+      }
+      cluster.getZkClient().create("/configs/_default/testdata", bytes, CreateMode.PERSISTENT, true);
+      Utils.executeGET(client.getHttpClient(),
+          basezk + "/configs/_default/testdata",
+          is -> {
+            byte[] newBytes = new byte[bytes.length];
+            is.read(newBytes);
+            for (int i = 0; i < newBytes.length; i++) {
+              assertEquals(bytes[i], newBytes[i]);
+            }
+            return null;
+          });
+    }
+  }
+
+}
\ No newline at end of file