You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ge...@apache.org on 2023/03/06 18:41:42 UTC

[solr] branch branch_9x updated: SOLR-16488: Migrate ZookeeperReadAPI to JAX-RS (#1358)

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

gerlowskija pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 82f5671ae70 SOLR-16488: Migrate ZookeeperReadAPI to JAX-RS (#1358)
82f5671ae70 is described below

commit 82f5671ae70325dec7573561205a871c367a5eb8
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Wed Mar 1 10:09:20 2023 -0500

    SOLR-16488: Migrate ZookeeperReadAPI to JAX-RS (#1358)
    
    ZookeeperReadAPI is now a JAX-RS based v2 API.
    
    This commit also updates the endpoints to match the cleaner, more REST-ful
    design for v2 that we want to use going forward:
    
    - "read ZK data" is now: GET /api/cluster/zookeeper/data/<path>
    - "get children" is now: GET /api/cluster/zookeeper/children/<path>
---
 solr/CHANGES.txt                                   |   4 +
 .../java/org/apache/solr/core/CoreContainer.java   |  13 +-
 .../solr/handler/admin/ZookeeperReadAPI.java       | 251 ++++++++++++++-------
 .../apache/solr/jersey/ExperimentalResponse.java   |  30 +++
 .../org/apache/solr/jersey/JerseyApplications.java |   1 +
 .../org/apache/solr/jersey/MessageBodyWriters.java |  15 ++
 .../apache/solr/response/RawResponseWriter.java    |  25 +-
 .../apache/solr/security/AuthorizationUtils.java   |   6 +-
 .../solr/handler/admin/ZookeeperReadAPITest.java   | 112 +++++++--
 .../solr/metrics/SolrMetricsIntegrationTest.java   |   4 +-
 10 files changed, 350 insertions(+), 111 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4bc8e23329a..ad10963d61f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -128,6 +128,10 @@ Improvements
   `GET /api/cores/coreName/snapshots`, `POST /api/cores/coreName/snapshots/snapshotName`, and
   `DELETE /api/cores/coreName/snapshots/snapshotName`, respectively  (John Durham via Jason Gerlowski)
 
+* SOLR-16488: The paths of the v2 "ZooKeeper read" APIs have been tweaked slightly to be more intuitive. "get node data" is now available
+  at `GET /api/cluster/zookeeper/data/<path>`, and "list node children" is now available at `GET /api/cluster/zookeeper/children/<path>`
+  (Jason Gerlowski, Joshua Ouma)
+
 Optimizations
 ---------------------
 
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 0e61a905dd6..ae8a62de115 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -69,6 +69,7 @@ import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.store.Directory;
 import org.apache.solr.api.ContainerPluginsRegistry;
+import org.apache.solr.api.JerseyResource;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
 import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder;
@@ -729,6 +730,14 @@ public class CoreContainer {
     containerHandlers.getApiBag().registerObject(apiObject);
   }
 
+  private void registerV2ApiIfEnabled(Class<? extends JerseyResource> clazz) {
+    if (containerHandlers.getJerseyEndpoints() == null) {
+      return;
+    }
+
+    containerHandlers.getJerseyEndpoints().register(clazz);
+  }
+
   // -------------------------------------------------------------------
   // Initialization / Cleanup
   // -------------------------------------------------------------------
@@ -798,9 +807,7 @@ public class CoreContainer {
       packageLoader = new SolrPackageLoader(this);
       registerV2ApiIfEnabled(packageLoader.getPackageAPI().editAPI);
       registerV2ApiIfEnabled(packageLoader.getPackageAPI().readAPI);
-
-      ZookeeperReadAPI zookeeperReadAPI = new ZookeeperReadAPI(this);
-      registerV2ApiIfEnabled(zookeeperReadAPI);
+      registerV2ApiIfEnabled(ZookeeperReadAPI.class);
     }
 
     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
index 139e3ba3520..a56c5b7deed 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperReadAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperReadAPI.java
@@ -17,32 +17,39 @@
 
 package org.apache.solr.handler.admin;
 
-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.SECURITY_READ_PERM;
 import static org.apache.solr.security.PermissionNameProvider.Name.ZK_READ_PERM;
 
-import java.io.IOException;
-import java.util.Collections;
+import com.fasterxml.jackson.annotation.JsonAnyGetter;
+import com.fasterxml.jackson.annotation.JsonAnySetter;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.swagger.v3.oas.annotations.Parameter;
 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 javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
 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.ContentStream;
 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.handler.admin.api.AdminAPIBase;
+import org.apache.solr.jersey.ExperimentalResponse;
+import org.apache.solr.jersey.JacksonReflectMapWriter;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.jersey.SolrJerseyResponse;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.RawResponseWriter;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
@@ -55,49 +62,73 @@ import org.apache.zookeeper.data.Stat;
  *
  * @lucene.experimental
  */
-public class ZookeeperReadAPI {
-  private final CoreContainer coreContainer;
-  private final SolrParams rawWtParams;
-
-  public ZookeeperReadAPI(CoreContainer coreContainer) {
-    this.coreContainer = coreContainer;
-    Map<String, String> map = new HashMap<>(1);
-    map.put(WT, "raw");
-    map.put(OMIT_HEADER, "true");
-    rawWtParams = new MapSolrParams(map);
+@Path("/cluster/zookeeper/")
+public class ZookeeperReadAPI extends AdminAPIBase {
+  @Inject
+  public ZookeeperReadAPI(
+      CoreContainer coreContainer, SolrQueryRequest req, SolrQueryResponse rsp) {
+    super(coreContainer, req, rsp);
   }
 
   /** Request contents of a znode, except security.json */
-  @EndPoint(path = "/cluster/zk/data/*", method = SolrRequest.METHOD.GET, permission = ZK_READ_PERM)
-  public void readNode(SolrQueryRequest req, SolrQueryResponse rsp) {
-    String path = req.getPathTemplateValues().get("*");
-    if (path == null || path.isEmpty()) path = "/";
-    readNodeAndAddToResponse(path, req, rsp);
+  @GET
+  @Path("/data{zkPath:.+}")
+  @Produces({RawResponseWriter.CONTENT_TYPE, MediaType.APPLICATION_JSON})
+  @PermissionName(ZK_READ_PERM)
+  public ZooKeeperFileResponse readNode(
+      @Parameter(description = "The path of the node to read from ZooKeeper") @PathParam("zkPath")
+          String zkPath) {
+    zkPath = sanitizeZkPath(zkPath);
+    return readNodeAndAddToResponse(zkPath);
   }
 
   /** Request contents of the security.json node */
-  @EndPoint(
-      path = "/cluster/zk/data/security.json",
-      method = SolrRequest.METHOD.GET,
-      permission = SECURITY_READ_PERM)
-  public void readSecurityJsonNode(SolrQueryRequest req, SolrQueryResponse rsp) {
-    String path = "/security.json";
-    readNodeAndAddToResponse(path, req, rsp);
+  @GET
+  @Path("/data/security.json")
+  @Produces({RawResponseWriter.CONTENT_TYPE, MediaType.APPLICATION_JSON})
+  @PermissionName(SECURITY_READ_PERM)
+  public ZooKeeperFileResponse readSecurityJsonNode() {
+    return readNodeAndAddToResponse("/security.json");
+  }
+
+  private String sanitizeZkPath(String zkPath) {
+    if (zkPath == null || zkPath.isEmpty()) {
+      return "/";
+    } else if (zkPath.length() > 1 && zkPath.endsWith("/")) {
+      return zkPath.substring(0, zkPath.length() - 1);
+    }
+
+    return zkPath;
   }
 
   /** List the children of a certain zookeeper znode */
-  @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 = "/";
+  @GET
+  @Path("/children{zkPath:.*}")
+  @Produces({"application/json", "application/javabin"})
+  @PermissionName(ZK_READ_PERM)
+  public ListZkChildrenResponse listNodes(
+      @Parameter(description = "The path of the ZooKeeper node to stat and list children of")
+          @PathParam("zkPath")
+          String zkPath,
+      @Parameter(
+              description =
+                  "Controls whether stat information for child nodes is included in the response. 'true' by default.")
+          @QueryParam("children")
+          Boolean includeChildren)
+      throws Exception {
+    final ListZkChildrenResponse listResponse =
+        instantiateJerseyResponse(ListZkChildrenResponse.class);
+
+    zkPath = sanitizeZkPath(zkPath);
     try {
-      Stat stat = coreContainer.getZkController().getZkClient().exists(path, null, true);
-      rsp.add("stat", (MapWriter) ew -> printStat(ew, stat));
-      if (!req.getParams().getBool("c", true)) {
-        return;
+      Stat stat = coreContainer.getZkController().getZkClient().exists(zkPath, null, true);
+      listResponse.stat = new AnnotatedStat(stat);
+      if (includeChildren != null && !includeChildren.booleanValue()) {
+        return listResponse;
       }
-      List<String> l = coreContainer.getZkController().getZkClient().getChildren(path, null, false);
-      String prefix = path.endsWith("/") ? path : path + "/";
+      List<String> l =
+          coreContainer.getZkController().getZkClient().getChildren(zkPath, null, false);
+      String prefix = zkPath.endsWith("/") ? zkPath : zkPath + "/";
 
       Map<String, Stat> stats = new LinkedHashMap<>();
       for (String s : l) {
@@ -108,23 +139,16 @@ public class ZookeeperReadAPI {
           throw new RuntimeException(e);
         }
       }
-      rsp.add(
-          path,
-          (MapWriter)
-              ew -> {
-                for (Map.Entry<String, Stat> e : stats.entrySet()) {
-                  ew.put(e.getKey(), (MapWriter) ew1 -> printStat(ew1, e.getValue()));
-                }
-              });
+
+      final Map<String, AnnotatedStat> childStats = new HashMap<>();
+      for (Map.Entry<String, Stat> e : stats.entrySet()) {
+        childStats.put(e.getKey(), new AnnotatedStat(e.getValue()));
+      }
+      listResponse.unknownFields.put(zkPath, childStats);
+
+      return listResponse;
     } 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);
+      throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "No such node :" + zkPath);
     }
   }
 
@@ -141,16 +165,19 @@ public class ZookeeperReadAPI {
     }
   }
 
-  /** Reads content of a znode and adds it to the response */
-  private void readNodeAndAddToResponse(
-      String zkPath, SolrQueryRequest req, SolrQueryResponse rsp) {
+  /** Reads content of a znode */
+  private ZooKeeperFileResponse readNodeAndAddToResponse(String zkPath) {
+    final ZooKeeperFileResponse zkFileResponse =
+        instantiateJerseyResponse(ZooKeeperFileResponse.class);
+
     byte[] d = readPathFromZookeeper(zkPath);
     if (d == null || d.length == 0) {
-      rsp.add(zkPath, null);
-      return;
+      zkFileResponse.zkData = EMPTY;
+      return zkFileResponse;
     }
-    req.setParams(SolrParams.wrapDefaults(rawWtParams, req.getParams()));
-    rsp.add(CONTENT, new ContentStreamBase.ByteArrayStream(d, null, guessMime(d[0])));
+
+    zkFileResponse.output = new ContentStreamBase.ByteArrayStream(d, null, guessMime(d[0]));
+    return zkFileResponse;
   }
 
   /** Reads a single node from zookeeper and return as byte array */
@@ -166,17 +193,85 @@ public class ZookeeperReadAPI {
     return d;
   }
 
-  private void printStat(MapWriter.EntryWriter ew, Stat stat) throws IOException {
-    ew.put("version", stat.getVersion());
-    ew.put("aversion", stat.getAversion());
-    ew.put("children", stat.getNumChildren());
-    ew.put("ctime", stat.getCtime());
-    ew.put("cversion", stat.getCversion());
-    ew.put("czxid", stat.getCzxid());
-    ew.put("ephemeralOwner", stat.getEphemeralOwner());
-    ew.put("mtime", stat.getMtime());
-    ew.put("mzxid", stat.getMzxid());
-    ew.put("pzxid", stat.getPzxid());
-    ew.put("dataLength", stat.getDataLength());
+  public static class ListZkChildrenResponse extends ExperimentalResponse {
+    @JsonProperty("stat")
+    public AnnotatedStat stat;
+
+    // TODO Currently the list response (when child information is fetched) consists primarily of an
+    //  object with only one key - the name of the root node - with separate objects under there for
+    //  each child.  The additional nesting under the root node doesn't serve much purpose afaict
+    //  and should be removed.
+    private Map<String, Map<String, AnnotatedStat>> unknownFields = new HashMap<>();
+
+    @JsonAnyGetter
+    public Map<String, Map<String, AnnotatedStat>> unknownProperties() {
+      return unknownFields;
+    }
+
+    @JsonAnySetter
+    public void setUnknownProperty(String field, Map<String, AnnotatedStat> value) {
+      unknownFields.put(field, value);
+    }
+  }
+
+  public static class AnnotatedStat implements JacksonReflectMapWriter {
+    @JsonProperty("version")
+    public int version;
+
+    @JsonProperty("aversion")
+    public int aversion;
+
+    @JsonProperty("children")
+    public int children;
+
+    @JsonProperty("ctime")
+    public long ctime;
+
+    @JsonProperty("cversion")
+    public int cversion;
+
+    @JsonProperty("czxid")
+    public long czxid;
+
+    @JsonProperty("ephemeralOwner")
+    public long ephemeralOwner;
+
+    @JsonProperty("mtime")
+    public long mtime;
+
+    @JsonProperty("mzxid")
+    public long mzxid;
+
+    @JsonProperty("pzxid")
+    public long pzxid;
+
+    @JsonProperty("dataLength")
+    public int dataLength;
+
+    public AnnotatedStat(Stat stat) {
+      this.version = stat.getVersion();
+      this.aversion = stat.getAversion();
+      this.children = stat.getNumChildren();
+      this.ctime = stat.getCtime();
+      this.cversion = stat.getCversion();
+      this.czxid = stat.getCzxid();
+      this.ephemeralOwner = stat.getEphemeralOwner();
+      this.mtime = stat.getMtime();
+      this.mzxid = stat.getMzxid();
+      this.pzxid = stat.getPzxid();
+      this.dataLength = stat.getDataLength();
+    }
+
+    public AnnotatedStat() {}
+  }
+
+  private static final String EMPTY = "empty";
+
+  public static class ZooKeeperFileResponse extends SolrJerseyResponse {
+    @JsonProperty(CONTENT) // A flag value that RawResponseWriter handles specially
+    public ContentStream output;
+
+    @JsonProperty("zkData")
+    public String zkData;
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/jersey/ExperimentalResponse.java b/solr/core/src/java/org/apache/solr/jersey/ExperimentalResponse.java
new file mode 100644
index 00000000000..657e3652064
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/jersey/ExperimentalResponse.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.jersey;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * {@link SolrJerseyResponse} implementation with a warning field indicating that the format may
+ * change
+ */
+public class ExperimentalResponse extends SolrJerseyResponse {
+  @JsonProperty("WARNING")
+  public String warning =
+      "This response format is experimental.  It is likely to change in the future.";
+}
diff --git a/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java b/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java
index ddeb81203ae..cb6b5bf0cde 100644
--- a/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java
+++ b/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java
@@ -55,6 +55,7 @@ public class JerseyApplications {
       register(MessageBodyWriters.JavabinMessageBodyWriter.class);
       register(MessageBodyWriters.XmlMessageBodyWriter.class);
       register(MessageBodyWriters.CsvMessageBodyWriter.class);
+      register(MessageBodyWriters.RawMessageBodyWriter.class);
       register(JacksonJsonProvider.class);
       register(SolrJacksonMapper.class);
 
diff --git a/solr/core/src/java/org/apache/solr/jersey/MessageBodyWriters.java b/solr/core/src/java/org/apache/solr/jersey/MessageBodyWriters.java
index ea9022cb38c..274f6b19fca 100644
--- a/solr/core/src/java/org/apache/solr/jersey/MessageBodyWriters.java
+++ b/solr/core/src/java/org/apache/solr/jersey/MessageBodyWriters.java
@@ -40,6 +40,7 @@ import org.apache.solr.response.BinaryResponseWriter;
 import org.apache.solr.response.CSVResponseWriter;
 import org.apache.solr.response.QueryResponseWriter;
 import org.apache.solr.response.QueryResponseWriterUtil;
+import org.apache.solr.response.RawResponseWriter;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.response.XMLResponseWriter;
 
@@ -79,6 +80,20 @@ public class MessageBodyWriters {
     }
   }
 
+  @Produces(RawResponseWriter.CONTENT_TYPE)
+  public static class RawMessageBodyWriter extends BaseMessageBodyWriter
+      implements MessageBodyWriter<JacksonReflectMapWriter> {
+    @Override
+    public QueryResponseWriter createResponseWriter() {
+      return new RawResponseWriter();
+    }
+
+    @Override
+    public String getSupportedMediaType() {
+      return RawResponseWriter.CONTENT_TYPE;
+    }
+  }
+
   @Produces(CONTENT_TYPE_TEXT_UTF8)
   public static class CsvMessageBodyWriter extends BaseMessageBodyWriter
       implements MessageBodyWriter<JacksonReflectMapWriter> {
diff --git a/solr/core/src/java/org/apache/solr/response/RawResponseWriter.java b/solr/core/src/java/org/apache/solr/response/RawResponseWriter.java
index 73404a4a7f3..eb4a12736e2 100644
--- a/solr/core/src/java/org/apache/solr/response/RawResponseWriter.java
+++ b/solr/core/src/java/org/apache/solr/response/RawResponseWriter.java
@@ -38,6 +38,8 @@ import org.apache.solr.request.SolrQueryRequest;
  * @since solr 1.3
  */
 public class RawResponseWriter implements BinaryQueryResponseWriter {
+
+  public static final String CONTENT_TYPE = "application/vnd.apache.solr.raw";
   /**
    * The key that should be used to add a ContentStream to the SolrQueryResponse if you intend to
    * use this Writer.
@@ -46,6 +48,14 @@ public class RawResponseWriter implements BinaryQueryResponseWriter {
 
   private String _baseWriter = null;
 
+  /**
+   * A fallback writer used for requests that don't return raw content and that aren't associated
+   * with a particular SolrCore
+   *
+   * <p>Populated upon first use.
+   */
+  private QueryResponseWriter defaultWriter;
+
   @Override
   public void init(NamedList<?> n) {
     if (n != null) {
@@ -56,9 +66,20 @@ public class RawResponseWriter implements BinaryQueryResponseWriter {
     }
   }
 
-  // Even if this is null, it should be ok
   protected QueryResponseWriter getBaseWriter(SolrQueryRequest request) {
-    return request.getCore().getQueryResponseWriter(_baseWriter);
+    if (request.getCore() != null) {
+      return request.getCore().getQueryResponseWriter(_baseWriter);
+    }
+
+    // Requests to a specific core already have writers, but we still need a 'default writer' for
+    // non-core
+    // (i.e. container-level) APIs
+    synchronized (this) {
+      if (defaultWriter == null) {
+        defaultWriter = new JSONResponseWriter();
+      }
+    }
+    return defaultWriter;
   }
 
   @Override
diff --git a/solr/core/src/java/org/apache/solr/security/AuthorizationUtils.java b/solr/core/src/java/org/apache/solr/security/AuthorizationUtils.java
index dda359f370d..6ea768aa69b 100644
--- a/solr/core/src/java/org/apache/solr/security/AuthorizationUtils.java
+++ b/solr/core/src/java/org/apache/solr/security/AuthorizationUtils.java
@@ -137,8 +137,10 @@ public class AuthorizationUtils {
   public static List<AuthorizationContext.CollectionRequest> getCollectionRequests(
       String path, List<String> collectionNames, SolrParams params) {
     final ArrayList<AuthorizationContext.CollectionRequest> collectionRequests = new ArrayList<>();
-    for (String collection : collectionNames) {
-      collectionRequests.add(new AuthorizationContext.CollectionRequest(collection));
+    if (collectionNames != null) {
+      for (String collection : collectionNames) {
+        collectionRequests.add(new AuthorizationContext.CollectionRequest(collection));
+      }
     }
 
     // Extract collection name from the params in case of a Collection Admin request
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
index bee96cb6d05..109e27d64d9 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperReadAPITest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperReadAPITest.java
@@ -19,33 +19,42 @@ package org.apache.solr.handler.admin;
 
 import static org.apache.solr.common.util.StrUtils.split;
 import static org.apache.solr.common.util.Utils.getObjectByPath;
+import static org.hamcrest.Matchers.containsInAnyOrder;
 
-import java.lang.invoke.MethodHandles;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import java.net.URL;
 import java.util.Map;
+import java.util.stream.Collectors;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.Utils;
 import org.apache.zookeeper.CreateMode;
+import org.hamcrest.MatcherAssert;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
+/** Integration tests for {@link ZookeeperReadAPI} */
 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();
   }
 
+  private URL baseUrl;
+  private String basezk;
+  private String basezkls;
+
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
+
+    baseUrl = cluster.getJettySolrRunner(0).getBaseUrl();
+    basezk = baseUrl.toString().replace("/solr", "/api") + "/cluster/zookeeper/data";
+    basezkls = baseUrl.toString().replace("/solr", "/api") + "/cluster/zookeeper/children";
   }
 
   @After
@@ -56,10 +65,6 @@ public class ZookeeperReadAPITest extends SolrCloudTestCase {
 
   @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);
@@ -73,27 +78,84 @@ public class ZookeeperReadAPITest extends SolrCloudTestCase {
       assertEquals("0", String.valueOf(getObjectByPath(o, true, split("/stat/version", '/'))));
 
       o = Utils.executeGET(client.getHttpClient(), basezk + "/configs", Utils.JSONCONSUMER);
-      assertTrue(((Map) o).containsKey("/configs"));
-      assertNull(((Map) o).get("/configs"));
+      assertTrue(((Map) o).containsKey("zkData"));
+      assertEquals("empty", ((Map) o).get("zkData"));
 
       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;
-          });
+      try {
+        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;
+            });
+      } finally {
+        cluster.getZkClient().delete("/configs/_default/testdata", -1, true);
+      }
+    }
+  }
+
+  @Test
+  public void testRequestingDataFromNonexistentNodeReturnsAnError() throws Exception {
+    try (HttpSolrClient client = new HttpSolrClient.Builder(baseUrl.toString()).build()) {
+      final SolrException expected =
+          expectThrows(
+              SolrException.class,
+              () -> {
+                Utils.executeGET(
+                    client.getHttpClient(),
+                    basezk + "/configs/_default/nonexistentnode",
+                    Utils.JSONCONSUMER);
+              });
+      assertEquals(404, expected.code());
+    }
+  }
+
+  @Test
+  public void testCanListChildNodes() throws Exception {
+    try (HttpSolrClient client = new HttpSolrClient.Builder(baseUrl.toString()).build()) {
+      final ZookeeperReadAPI.ListZkChildrenResponse response =
+          Utils.executeGET(
+              client.getHttpClient(),
+              basezkls + "/configs/_default",
+              is -> {
+                return new ObjectMapper()
+                    .readValue(is, ZookeeperReadAPI.ListZkChildrenResponse.class);
+              });
+
+      // At the top level, the response contains a key with the value of the specified zkPath
+      assertEquals(1, response.unknownProperties().size());
+      assertEquals(
+          "/configs/_default",
+          response.unknownProperties().keySet().stream().collect(Collectors.toList()).get(0));
+
+      // Under the specified zkPath is a key for each child, with values being that stat for that
+      // node.
+      // The actual stat values vary a good bit so aren't very useful to assert on, so let's just
+      // make sure all of the expected child nodes were found.
+      final Map<String, ZookeeperReadAPI.AnnotatedStat> childStatsByPath =
+          response.unknownProperties().get("/configs/_default");
+      assertEquals(6, childStatsByPath.size());
+      MatcherAssert.assertThat(
+          childStatsByPath.keySet(),
+          containsInAnyOrder(
+              "protwords.txt",
+              "solrconfig.xml",
+              "synonyms.txt",
+              "stopwords.txt",
+              "managed-schema.xml",
+              "lang"));
     }
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
index 25cc82c0bb2..7f77a701d05 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
@@ -260,7 +260,9 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
         assertNotNull(zkMmetrics.get(k));
       }
       Utils.executeGET(
-          httpClient, j.getBaseURLV2() + "/cluster/zk/ls/live_nodes", Utils.JSONCONSUMER);
+          httpClient,
+          j.getBaseURLV2() + "/cluster/zookeeper/children/live_nodes",
+          Utils.JSONCONSUMER);
       @SuppressWarnings("unchecked")
       Map<String, Object> zkMmetricsNew =
           (Map<String, Object>)