diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 2fec97d6129..8476a6b24bb 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -199,7 +199,9 @@ Optimizations that which consumes almost no memory, saving 1MB of memory per SolrCore. (David Smiley) * SOLR-17381: Make CLUSTERSTATUS request configurable to improve performance by allowing retrieval of specific information, - reducing unnecessary data fetching. (Aparna Suresh, David Smiley) + reducing unnecessary data fetching. Enhanced CloudSolrClient's HTTP ClusterStateProvider to use it, and to scale to + more collections better as well. + (Aparna Suresh, David Smiley) * SOLR-17396: Reduce thread contention in ZkStateReader.getCollectionProperties(). (Aparna Suresh, David Smiley, Paul McArthur) diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java index 1c70170608f..b8c85314966 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java @@ -23,16 +23,18 @@ import java.lang.invoke.MethodHandles; import java.time.Instant; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.client.solrj.SolrRequest.METHOD; import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.request.CollectionAdminRequest; -import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.client.solrj.request.GenericSolrRequest; import org.apache.solr.client.solrj.response.CollectionAdminResponse; import org.apache.solr.common.cloud.Aliases; import org.apache.solr.common.cloud.ClusterState; @@ -128,37 +130,28 @@ public ClusterState.CollectionRef getState(String collection) { } @SuppressWarnings("unchecked") - private ClusterState fetchClusterState(SolrClient client, Map clusterProperties) + private ClusterState fetchClusterState(SolrClient client) throws SolrServerException, IOException, NotACollectionException { SimpleOrderedMap cluster = submitClusterStateRequest(client, null, ClusterStateRequestType.FETCH_CLUSTER_STATE); - Map collectionsMap = ((NamedList) cluster.get("collections")).asMap(10); - - ClusterState cs = new ClusterState(this.liveNodes, new HashMap<>()); List liveNodesList = (List) cluster.get("live_nodes"); if (liveNodesList != null) { - Set liveNodes = new HashSet<>(liveNodesList); - this.liveNodes = liveNodes; + this.liveNodes = Set.copyOf(liveNodesList); liveNodesTimestamp = System.nanoTime(); - cs = new ClusterState(liveNodes, new HashMap<>()); } - for (Map.Entry e : collectionsMap.entrySet()) { - String collectionName = e.getKey(); - Map collStateMap = (Map) e.getValue(); - cs = - cs.copyWith( - collectionName, getDocCollectionFromObjects(collectionName, collStateMap, -1)); - } + var collectionsNl = (NamedList>) cluster.get("collections"); - if (clusterProperties != null) { - Map properties = (Map) cluster.get("properties"); - if (properties != null) { - clusterProperties.putAll(properties); - } + Map collStateByName = new LinkedHashMap<>(collectionsNl.size()); + for (Entry> entry : collectionsNl) { + final var collStateMap = entry.getValue().asMap(10); + final int zNodeVersion = (int) collStateMap.get("znodeVersion"); + collStateByName.put( + entry.getKey(), getDocCollectionFromObjects(entry.getKey(), collStateMap, zNodeVersion)); } - return cs; + + return new ClusterState(this.liveNodes, collStateByName); } private DocCollection getDocCollectionFromObjects( @@ -197,23 +190,18 @@ private SimpleOrderedMap submitClusterStateRequest( ModifiableSolrParams params = new ModifiableSolrParams(); params.set("action", "CLUSTERSTATUS"); - if (requestType == ClusterStateRequestType.FETCH_COLLECTION && collection != null) { - params.set("collection", collection); - } else if (requestType == ClusterStateRequestType.FETCH_LIVE_NODES) { - params.set("liveNodes", "true"); - } else if (requestType == ClusterStateRequestType.FETCH_CLUSTER_PROP) { - params.set("clusterProperties", "true"); - } else if (requestType == ClusterStateRequestType.FETCH_NODE_ROLES) { - params.set("roles", "true"); - } - if (requestType == ClusterStateRequestType.FETCH_CLUSTER_STATE) { - params.set("includeAll", "true"); - } else { - params.set("includeAll", "false"); + params.set("includeAll", false); // will flip flor CLUSTER_STATE + switch (requestType) { + case FETCH_CLUSTER_STATE -> params.set("includeAll", true); + case FETCH_COLLECTION -> { + if (collection != null) params.set("collection", collection); + } + case FETCH_LIVE_NODES -> params.set("liveNodes", true); + case FETCH_CLUSTER_PROP -> params.set("clusterProperties", true); + case FETCH_NODE_ROLES -> params.set("roles", true); } - params.set("prs", "true"); - QueryRequest request = new QueryRequest(params); - request.setPath("/admin/collections"); + params.set("prs", true); + var request = new GenericSolrRequest(METHOD.GET, "/admin/collections", params); return (SimpleOrderedMap) client.request(request).get("cluster"); } @@ -359,7 +347,7 @@ public ClusterState getClusterState() { for (String nodeName : liveNodes) { String baseUrl = Utils.getBaseUrlForNodeName(nodeName, urlScheme); try (SolrClient client = getSolrClient(baseUrl)) { - return fetchClusterState(client, null); + return fetchClusterState(client); } catch (SolrServerException | RemoteSolrException | IOException e) { log.warn("Attempt to fetch cluster state from {} failed.", baseUrl, e); } catch (NotACollectionException e) {