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

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


The following commit(s) were added to refs/heads/main by this push:
     new d45b86b1332 SOLR-17931: Remove deprecated methods in collection state 
classes (#3834)
d45b86b1332 is described below

commit d45b86b13326ad0434f22187e454ceb217e21e88
Author: Pierre Salagnac <[email protected]>
AuthorDate: Thu Nov 13 14:28:51 2025 +0100

    SOLR-17931: Remove deprecated methods in collection state classes (#3834)
    
    This removes some deprecated methods from DocCollection and related 
classes. Callers are adapted to use other similar methods when appropriate.
    Goal is to remove some deprecated code from Solr 10.
---
 .../SOLR-17931-remove-clusterstate-methods.yml     |   8 ++
 .../java/org/apache/solr/cloud/ConfigSetCmds.java  |   3 +-
 .../java/org/apache/solr/cloud/ZkController.java   |  18 +++-
 .../apache/solr/cloud/overseer/NodeMutator.java    |   2 +-
 .../solr/handler/designer/SchemaDesignerAPI.java   |   7 +-
 .../org/apache/solr/schema/ManagedIndexSchema.java |   5 +-
 .../solr/search/join/ScoreJoinQParserPlugin.java   |   2 +-
 .../java/org/apache/solr/servlet/HttpSolrCall.java |   2 +-
 .../processor/DistributedZkUpdateProcessor.java    |   6 +-
 .../DocExpirationUpdateProcessorFactory.java       |  12 +--
 .../processor/RoutedAliasUpdateProcessor.java      |  19 ++--
 .../apache/solr/cloud/ClusterStateMockUtil.java    |   8 +-
 .../apache/solr/cloud/CreateRoutedAliasTest.java   |   4 +-
 .../test/org/apache/solr/cloud/DeleteNodeTest.java |  32 +++---
 .../org/apache/solr/cloud/MigrateReplicasTest.java |  24 ++---
 .../org/apache/solr/cloud/ReplaceNodeTest.java     |  17 +---
 .../test/org/apache/solr/cloud/SliceStateTest.java |  10 +-
 .../org/apache/solr/cloud/TestHashPartitioner.java |   8 +-
 .../test/org/apache/solr/core/CoreSorterTest.java  |   8 +-
 .../solr/client/solrj/io/sql/StatementImpl.java    |   2 +-
 .../client/solrj/io/stream/CloudSolrStream.java    |  10 +-
 .../solrj/io/stream/FeaturesSelectionStream.java   |   2 +-
 .../client/solrj/io/stream/TextLogitStream.java    |   2 +-
 .../solr/client/solrj/io/stream/TopicStream.java   |   6 +-
 .../solr/client/solrj/io/stream/TupleStream.java   |   2 +-
 .../solr/client/solrj/impl/CloudSolrClient.java    |   4 +-
 .../org/apache/solr/common/cloud/ClusterState.java |  92 +-----------------
 .../solr/common/cloud/CompositeIdRouter.java       |   2 +-
 .../apache/solr/common/cloud/DocCollection.java    | 107 +--------------------
 .../apache/solr/common/cloud/HashBasedRouter.java  |   2 +-
 .../java/org/apache/solr/common/cloud/Replica.java |  20 +---
 .../cloud/PerReplicaStatesIntegrationTest.java     |   4 +-
 .../AbstractCloudBackupRestoreTestCase.java        |   7 +-
 33 files changed, 139 insertions(+), 318 deletions(-)

diff --git a/changelog/unreleased/SOLR-17931-remove-clusterstate-methods.yml 
b/changelog/unreleased/SOLR-17931-remove-clusterstate-methods.yml
new file mode 100644
index 00000000000..585239f9e81
--- /dev/null
+++ b/changelog/unreleased/SOLR-17931-remove-clusterstate-methods.yml
@@ -0,0 +1,8 @@
+# See https://github.com/apache/solr/blob/main/dev-docs/changelog.adoc
+title: Remove deprecated methods in collection state classes
+type: removed # added, changed, fixed, deprecated, removed, dependency_update, 
security, other
+authors:
+  - name: Pierre Salagnac
+links:
+  - name: SOLR-17931
+  - url: https://issues.apache.org/jira/browse/SOLR-17931
diff --git a/solr/core/src/java/org/apache/solr/cloud/ConfigSetCmds.java 
b/solr/core/src/java/org/apache/solr/cloud/ConfigSetCmds.java
index 52219bbc2ea..258f47000f6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ConfigSetCmds.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ConfigSetCmds.java
@@ -185,7 +185,8 @@ public class ConfigSetCmds {
 
     zkStateReader
         .getClusterState()
-        .forEachCollection(
+        .collectionStream()
+        .forEach(
             state -> {
               String configName = state.getConfigName();
               if (configSetName.equals(configName))
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java 
b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 4cb6a598432..e0ed4912a83 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -55,6 +55,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import org.apache.curator.framework.api.ACLProvider;
 import org.apache.solr.client.api.util.SolrVersion;
@@ -1193,8 +1194,7 @@ public class ZkController implements Closeable {
           (collectionState) -> {
             if (collectionState == null) return false;
             boolean allStatesCorrect =
-                
Optional.ofNullable(collectionState.getReplicasOnNode(nodeName)).stream()
-                    .flatMap(List::stream)
+                collectionState.getReplicasOnNode(nodeName).stream()
                     .allMatch(replica -> replica.getState() == 
Replica.State.DOWN);
 
             if (allStatesCorrect
@@ -2069,15 +2069,25 @@ public class ZkController implements Closeable {
     if (log.isDebugEnabled()) {
       log.debug("waiting to find shard id in clusterstate for {}", 
cd.getName());
     }
+
+    Predicate<Replica> replicaPredicate = (Replica r) -> 
cd.getName().equals(r.getCoreName());
     try {
       DocCollection collection =
           zkStateReader.waitForState(
               cd.getCollectionName(),
               320,
               TimeUnit.SECONDS,
-              c -> c != null && c.getShardId(getNodeName(), cd.getName()) != 
null);
+              c ->
+                  c != null
+                      && 
c.getReplicasOnNode(getNodeName()).stream().anyMatch(replicaPredicate));
       // Read outside the predicate to avoid multiple potential writes
-      cd.getCloudDescriptor().setShardId(collection.getShardId(getNodeName(), 
cd.getName()));
+      String shardId =
+          collection.getReplicasOnNode(getNodeName()).stream()
+              .filter(replicaPredicate)
+              .map(Replica::getShard)
+              .findFirst()
+              .orElseThrow();
+      cd.getCloudDescriptor().setShardId(shardId);
     } catch (TimeoutException | InterruptedException e) {
       SolrZkClient.checkInterrupted(e);
       throw new SolrException(
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java 
b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
index bd45c3e8ea2..d92722c539b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
@@ -72,7 +72,7 @@ public class NodeMutator {
     final Map<String, Slice> slicesCopy = new 
LinkedHashMap<>(docCollection.getSlicesMap());
 
     List<Replica> replicasOnNode = docCollection.getReplicasOnNode(nodeName);
-    if (replicasOnNode == null || replicasOnNode.isEmpty()) {
+    if (replicasOnNode.isEmpty()) {
       return Optional.empty();
     }
     for (Replica replica : replicasOnNode) {
diff --git 
a/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerAPI.java 
b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerAPI.java
index d5a1a8b8d11..b7f7521ba48 100644
--- a/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerAPI.java
@@ -33,6 +33,7 @@ import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -60,6 +61,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkMaintenanceUtils;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -1119,8 +1121,9 @@ public class SchemaDesignerAPI implements 
SchemaDesignerConstants {
     Map<String, Object> response = new HashMap<>();
 
     DocCollection coll = zkStateReader().getCollection(mutableId);
-    if (coll.getActiveSlicesArr().length > 0) {
-      String coreName = coll.getActiveSlicesArr()[0].getLeader().getCoreName();
+    Collection<Slice> activeSlices = coll.getActiveSlices();
+    if (!activeSlices.isEmpty()) {
+      String coreName = 
activeSlices.stream().findAny().orElseThrow().getLeader().getCoreName();
       response.put("core", coreName);
     }
 
diff --git a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java 
b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
index 053427b9eb3..671aafe0555 100644
--- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
@@ -327,9 +327,8 @@ public final class ManagedIndexSchema extends IndexSchema {
     ClusterState clusterState = zkStateReader.getClusterState();
     Set<String> liveNodes = clusterState.getLiveNodes();
     final DocCollection docCollection = 
clusterState.getCollectionOrNull(collection);
-    if (docCollection != null && docCollection.getActiveSlicesArr().length > 
0) {
-      final Slice[] activeSlices = docCollection.getActiveSlicesArr();
-      for (Slice next : activeSlices) {
+    if (docCollection != null) {
+      for (Slice next : docCollection.getActiveSlices()) {
         Map<String, Replica> replicasMap = next.getReplicasMap();
         if (replicasMap != null) {
           for (Map.Entry<String, Replica> entry : replicasMap.entrySet()) {
diff --git 
a/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java 
b/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java
index b8f3de40f23..d049a4a9f01 100644
--- a/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java
@@ -496,7 +496,7 @@ public class ScoreJoinQParserPlugin extends QParserPlugin {
     final String nodeName = zkController.getNodeName();
     String fromReplica = null;
 
-    for (Slice slice : fromCollection.getActiveSlicesArr()) {
+    for (Slice slice : fromCollection.getActiveSlices()) {
       assert fromReplica == null;
       for (Replica replica : slice.getReplicas()) {
         if (replica.getNodeName().equals(nodeName)) {
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java 
b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 24ec79d1e5f..417467a9760 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -889,7 +889,7 @@ public class HttpSolrCall {
 
     if (isPreferLeader) {
       SolrCore core = null;
-      if (replicas != null && !replicas.isEmpty()) {
+      if (!replicas.isEmpty()) {
         List<Replica> leaderReplicas = 
replicas.stream().filter(Replica::isLeader).toList();
         core = randomlyGetSolrCore(liveNodes, leaderReplicas);
         if (core != null) return core;
diff --git 
a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
 
b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
index f744be22755..df0c8f98a2e 100644
--- 
a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
+++ 
b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
@@ -984,9 +984,9 @@ public class DistributedZkUpdateProcessor extends 
DistributedUpdateProcessor {
           for (Map.Entry<String, RoutingRule> entry : routingRules.entrySet()) 
{
             String targetCollectionName = 
entry.getValue().getTargetCollectionName();
             final DocCollection docCollection = 
cstate.getCollectionOrNull(targetCollectionName);
-            if (docCollection != null && 
docCollection.getActiveSlicesArr().length > 0) {
-              final Slice[] activeSlices = docCollection.getActiveSlicesArr();
-              Slice any = activeSlices[0];
+            if (docCollection != null && 
!docCollection.getActiveSlices().isEmpty()) {
+              final Collection<Slice> activeSlices = 
docCollection.getActiveSlices();
+              Slice any = activeSlices.stream().findAny().orElseThrow();
               if (nodes == null) nodes = new ArrayList<>();
               nodes.add(new SolrCmdDistributor.StdNode(new 
ZkCoreNodeProps(any.getLeader())));
             }
diff --git 
a/solr/core/src/java/org/apache/solr/update/processor/DocExpirationUpdateProcessorFactory.java
 
b/solr/core/src/java/org/apache/solr/update/processor/DocExpirationUpdateProcessorFactory.java
index e859a327d77..827e3f33381 100644
--- 
a/solr/core/src/java/org/apache/solr/update/processor/DocExpirationUpdateProcessorFactory.java
+++ 
b/solr/core/src/java/org/apache/solr/update/processor/DocExpirationUpdateProcessorFactory.java
@@ -22,11 +22,9 @@ import static 
org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.RejectedExecutionHandler;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -464,13 +462,13 @@ public final class DocExpirationUpdateProcessorFactory 
extends UpdateRequestProc
     String col = desc.getCollectionName();
 
     DocCollection docCollection = zk.getClusterState().getCollection(col);
-    if (docCollection.getActiveSlicesArr().length == 0) {
+    Optional<Slice> firstSlice =
+        docCollection.getActiveSlices().stream().min(COMPARE_SLICES_BY_NAME);
+    if (firstSlice.isEmpty()) {
       log.error("Collection {} has no active Slices?", col);
       return false;
     }
-    List<Slice> slices = new 
ArrayList<>(Arrays.asList(docCollection.getActiveSlicesArr()));
-    slices.sort(COMPARE_SLICES_BY_NAME);
-    Replica firstSliceLeader = slices.get(0).getLeader();
+    Replica firstSliceLeader = firstSlice.get().getLeader();
     if (null == firstSliceLeader) {
       log.warn("Slice in charge of periodic deletes for {} does not currently 
have a leader", col);
       return false;
diff --git 
a/solr/core/src/java/org/apache/solr/update/processor/RoutedAliasUpdateProcessor.java
 
b/solr/core/src/java/org/apache/solr/update/processor/RoutedAliasUpdateProcessor.java
index 0cbb1f71b23..da5e771e2e8 100644
--- 
a/solr/core/src/java/org/apache/solr/update/processor/RoutedAliasUpdateProcessor.java
+++ 
b/solr/core/src/java/org/apache/solr/update/processor/RoutedAliasUpdateProcessor.java
@@ -22,6 +22,7 @@ import static 
org.apache.solr.update.processor.DistributingUpdateProcessorFactor
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -266,13 +267,17 @@ public class RoutedAliasUpdateProcessor extends 
UpdateRequestProcessor {
   }
 
   private SolrCmdDistributor.Node lookupShardLeaderOfCollection(String 
collection) {
-    final Slice[] activeSlices =
-        
zkController.getClusterState().getCollection(collection).getActiveSlicesArr();
-    if (activeSlices.length == 0) {
-      throw new SolrException(
-          SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Cannot route to 
collection " + collection);
-    }
-    final Slice slice = activeSlices[0];
+    final Collection<Slice> activeSlices =
+        
zkController.getClusterState().getCollection(collection).getActiveSlices();
+    final Slice slice =
+        activeSlices.stream()
+            .findAny()
+            .orElseThrow(
+                () ->
+                    new SolrException(
+                        SolrException.ErrorCode.SERVICE_UNAVAILABLE,
+                        "Cannot route to collection " + collection));
+
     return getLeaderNode(collection, slice);
   }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java 
b/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java
index 51a3c3263f5..5076e981f3e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java
@@ -18,6 +18,7 @@
 package org.apache.solr.cloud;
 
 import java.nio.charset.StandardCharsets;
+import java.time.Instant;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -129,11 +130,14 @@ public class ClusterStateMockUtil {
           collectionProps.put(
               ZkStateReader.CONFIGNAME_PROP, 
ConfigSetsHandler.DEFAULT_CONFIGSET_NAME);
           docCollection =
-              new DocCollection(
+              DocCollection.create(
                   collName = "collection" + (collectionStates.size() + 1),
                   slices,
                   collectionProps,
-                  DocRouter.DEFAULT);
+                  DocRouter.DEFAULT,
+                  Integer.MAX_VALUE,
+                  Instant.EPOCH,
+                  null);
           collectionStates.put(docCollection.getName(), docCollection);
           break;
         case "s":
diff --git 
a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java 
b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
index 2aa959842ce..0972c7abba9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
@@ -156,8 +156,8 @@ public class CreateRoutedAliasTest extends 
SolrCloudTestCase {
             .sum());
     // assertEquals(1, coll.getNumNrtReplicas().intValue()); // TODO seems to 
be erroneous; I
     // figured 'null'
-    assertEquals(1, coll.getNumTlogReplicas().intValue()); // per-shard
-    assertEquals(1, coll.getNumPullReplicas().intValue()); // per-shard
+    assertEquals(1, coll.getNumReplicas(Replica.Type.TLOG)); // per-shard
+    assertEquals(1, coll.getNumReplicas(Replica.Type.PULL)); // per-shard
     assertTrue(
         "nodeSet didn't work?",
         coll.getSlices().stream()
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java 
b/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java
index 527ac45dfa1..955ef9ca5a6 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java
@@ -29,7 +29,6 @@ import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.StrUtils;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -72,27 +71,24 @@ public class DeleteNodeTest extends SolrCloudTestCase {
     DocCollection docColl = state.getCollection(coll);
     log.info("#### DocCollection: {}", docColl);
     List<Replica> replicas = docColl.getReplicasOnNode(nodeToBeDecommissioned);
-    if (replicas != null) {
-      for (Replica replica : replicas) {
-        String shard =
-            docColl.getShardId(
-                nodeToBeDecommissioned, 
replica.getStr(ZkStateReader.CORE_NAME_PROP));
-        Slice slice = docColl.getSlice(shard);
-        boolean hasOtherNonPullReplicas = false;
-        for (Replica r : slice.getReplicas()) {
-          if (!r.getName().equals(replica.getName())
-              && !r.getNodeName().equals(nodeToBeDecommissioned)
-              && r.getType() != Replica.Type.PULL) {
-            hasOtherNonPullReplicas = true;
-            break;
-          }
-        }
-        if (!hasOtherNonPullReplicas) {
-          shouldFail = true;
+    for (Replica replica : replicas) {
+      String shard = replica.getShard();
+      Slice slice = docColl.getSlice(shard);
+      boolean hasOtherNonPullReplicas = false;
+      for (Replica r : slice.getReplicas()) {
+        if (!r.getName().equals(replica.getName())
+            && !r.getNodeName().equals(nodeToBeDecommissioned)
+            && r.getType() != Replica.Type.PULL) {
+          hasOtherNonPullReplicas = true;
           break;
         }
       }
+      if (!hasOtherNonPullReplicas) {
+        shouldFail = true;
+        break;
+      }
     }
+
     new 
CollectionAdminRequest.DeleteNode(nodeToBeDecommissioned).processAsync("003", 
cloudClient);
     CollectionAdminRequest.RequestStatus requestStatus =
         CollectionAdminRequest.requestStatus("003");
diff --git a/solr/core/src/test/org/apache/solr/cloud/MigrateReplicasTest.java 
b/solr/core/src/test/org/apache/solr/cloud/MigrateReplicasTest.java
index 80a8377e791..610ba8aa314 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MigrateReplicasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MigrateReplicasTest.java
@@ -28,7 +28,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.stream.Collectors;
 import org.apache.http.HttpEntity;
 import org.apache.http.client.methods.HttpEntityEnclosingRequestBase;
 import org.apache.http.client.methods.HttpPost;
@@ -145,9 +144,9 @@ public class MigrateReplicasTest extends SolrCloudTestCase {
     collection = cloudClient.getClusterState().getCollectionOrNull(coll, 
false);
     log.debug("### After decommission: {}", collection);
     // check what are replica states on the decommissioned node
-    assertNull(
+    assertTrue(
         "There should not be any replicas left on decommissioned node",
-        collection.getReplicasOnNode(nodeToBeDecommissioned));
+        collection.getReplicasOnNode(nodeToBeDecommissioned).isEmpty());
 
     // let's do it back - this time wait for recoveries
     response =
@@ -184,17 +183,14 @@ public class MigrateReplicasTest extends 
SolrCloudTestCase {
     }
     // make sure all newly created replicas on node are active
     List<Replica> newReplicas = 
collection.getReplicasOnNode(nodeToBeDecommissioned);
-    assertNotNull("There should be replicas on the migrated-to node", 
newReplicas);
     assertFalse("There should be replicas on the migrated-to node", 
newReplicas.isEmpty());
     for (Replica r : newReplicas) {
       assertEquals(r.toString(), Replica.State.ACTIVE, r.getState());
     }
     // make sure all replicas on emptyNode are not active
     List<Replica> replicas = collection.getReplicasOnNode(emptyNode);
-    if (replicas != null) {
-      for (Replica r : replicas) {
-        assertNotEquals(r.toString(), Replica.State.ACTIVE, r.getState());
-      }
+    for (Replica r : replicas) {
+      assertNotEquals(r.toString(), Replica.State.ACTIVE, r.getState());
     }
 
     // check replication metrics on this jetty - see SOLR-14924
@@ -261,10 +257,6 @@ public class MigrateReplicasTest extends SolrCloudTestCase 
{
 
     DocCollection initialCollection = 
cloudClient.getClusterState().getCollection(coll);
     log.info("### Before decommission: {}", initialCollection);
-    List<Integer> initialReplicaCounts =
-        l.stream()
-            .map(node -> initialCollection.getReplicasOnNode(node).size())
-            .collect(Collectors.toList());
     Map<?, ?> response =
         callMigrateReplicas(
             cloudClient,
@@ -281,13 +273,9 @@ public class MigrateReplicasTest extends SolrCloudTestCase 
{
     // check what are replica states on the decommissioned nodes
     for (String nodeToBeDecommissioned : nodesToBeDecommissioned) {
       List<Replica> replicas = 
collection.getReplicasOnNode(nodeToBeDecommissioned);
-      if (replicas == null) {
-        replicas = Collections.emptyList();
-      }
-      assertEquals(
+      assertTrue(
           "There should be no more replicas on the sourceNode after a 
migrateReplicas request.",
-          Collections.emptyList(),
-          replicas);
+          replicas.isEmpty());
     }
 
     for (String node : eventualTargetNodes) {
diff --git a/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java 
b/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java
index e34d8bf6fda..bc64635d2d8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ReplaceNodeTest.java
@@ -128,9 +128,6 @@ public class ReplaceNodeTest extends SolrCloudTestCase {
     log.debug("### After decommission: {}", collection);
     // check what are replica states on the decommissioned node
     List<Replica> replicas = 
collection.getReplicasOnNode(nodeToBeDecommissioned);
-    if (replicas == null) {
-      replicas = Collections.emptyList();
-    }
     log.debug("### Existing replicas on decommissioned node: {}", replicas);
 
     // let's do it back - this time wait for recoveries
@@ -170,10 +167,8 @@ public class ReplaceNodeTest extends SolrCloudTestCase {
     }
     // make sure all replicas on emptyNode are not active
     replicas = collection.getReplicasOnNode(emptyNode);
-    if (replicas != null) {
-      for (Replica r : replicas) {
-        assertNotEquals(r.toString(), Replica.State.ACTIVE, r.getState());
-      }
+    for (Replica r : replicas) {
+      assertNotEquals(r.toString(), Replica.State.ACTIVE, r.getState());
     }
 
     // check replication metrics on this jetty - see SOLR-14924
@@ -256,13 +251,9 @@ public class ReplaceNodeTest extends SolrCloudTestCase {
     log.debug("### After decommission: {}", collection);
     // check what are replica states on the decommissioned node
     List<Replica> replicas = 
collection.getReplicasOnNode(nodeToBeDecommissioned);
-    if (replicas == null) {
-      replicas = Collections.emptyList();
-    }
-    assertEquals(
+    assertTrue(
         "There should be no more replicas on the sourceNode after a 
replaceNode request.",
-        Collections.emptyList(),
-        replicas);
+        replicas.isEmpty());
     int sizeA = collection.getReplicasOnNode(emptyNodes.get(0)).size();
     int sizeB = collection.getReplicasOnNode(emptyNodes.get(1)).size();
     assertEquals(
diff --git a/solr/core/src/test/org/apache/solr/cloud/SliceStateTest.java 
b/solr/core/src/test/org/apache/solr/cloud/SliceStateTest.java
index 64ec9f3c035..6b5fe92a519 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SliceStateTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SliceStateTest.java
@@ -56,7 +56,15 @@ public class SliceStateTest extends SolrTestCaseJ4 {
     assertSame("Default state not set to active", Slice.State.ACTIVE, 
slice.getState());
     slices.put("shard1", slice);
     collectionStates.put(
-        "collection1", new DocCollection("collection1", slices, props, 
DocRouter.DEFAULT));
+        "collection1",
+        DocCollection.create(
+            "collection1",
+            slices,
+            props,
+            DocRouter.DEFAULT,
+            Integer.MAX_VALUE,
+            Instant.EPOCH,
+            null));
 
     ClusterState clusterState = new ClusterState(liveNodes, collectionStates);
     byte[] bytes = Utils.toJSON(clusterState);
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java 
b/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java
index af09e728a00..9b9dd28c7aa 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.cloud;
 
+import java.time.Instant;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -295,12 +296,15 @@ public class TestHashPartitioner extends SolrTestCaseJ4 {
       slices.put(slice.getName(), slice);
     }
 
-    return new DocCollection(
+    return DocCollection.create(
         "collection1",
         slices,
         Collections.singletonMap(
             ZkStateReader.CONFIGNAME_PROP, 
ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
-        router);
+        router,
+        Integer.MAX_VALUE,
+        Instant.EPOCH,
+        null);
   }
 
   // from negative to positive, the upper bits of the hash ranges should be
diff --git a/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java 
b/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
index bbb55825445..48ea5d3b44a 100644
--- a/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
+++ b/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
@@ -19,6 +19,7 @@ package org.apache.solr.core;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.time.Instant;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -141,12 +142,15 @@ public class CoreSorterTest extends SolrTestCaseJ4 {
       }
       @SuppressWarnings({"unchecked"})
       DocCollection col =
-          new DocCollection(
+          DocCollection.create(
               collection,
               sliceMap,
               Collections.singletonMap(
                   ZkStateReader.CONFIGNAME_PROP, 
ConfigSetsHandler.DEFAULT_CONFIGSET_NAME),
-              DocRouter.DEFAULT);
+              DocRouter.DEFAULT,
+              Integer.MAX_VALUE,
+              Instant.EPOCH,
+              null);
       collToState.put(collection, col);
     }
     // reverse map
diff --git 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java
 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java
index f76f7bdb06e..f0ff025eda4 100644
--- 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java
+++ 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java
@@ -74,7 +74,7 @@ class StatementImpl implements Statement {
 
   protected SolrStream constructStream(String sql) throws IOException {
     try {
-      Slice[] slices =
+      List<Slice> slices =
           CloudSolrStream.getSlices(
               this.connection.getCollection(), this.connection.getClient(), 
true);
 
diff --git 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
index 84d34f99839..9c1277aed33 100644
--- 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
+++ 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
@@ -348,7 +348,7 @@ public class CloudSolrStream extends TupleStream implements 
Expressible {
     }
   }
 
-  public static Slice[] getSlices(
+  public static List<Slice> getSlices(
       String collectionName, CloudSolrClient cloudSolrClient, boolean 
checkAlias)
       throws IOException {
 
@@ -364,13 +364,13 @@ public class CloudSolrStream extends TupleStream 
implements Expressible {
 
     // Lookup all actives slices for these collections
     ClusterState clusterState = cloudSolrClient.getClusterState();
-    Slice[] slices =
+    List<Slice> slices =
         allCollections
             .map(c -> clusterState.getCollectionOrNull(c, true))
             .filter(Objects::nonNull)
-            .flatMap(docCol -> Arrays.stream(docCol.getActiveSlicesArr()))
-            .toArray(Slice[]::new);
-    if (slices.length == 0) {
+            .flatMap(docCol -> docCol.getActiveSlices().stream())
+            .toList();
+    if (slices.isEmpty()) {
       throw new IOException("Slices not found for " + collectionName);
     }
     return slices;
diff --git 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java
 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java
index 946a1141f17..ecae4363206 100644
--- 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java
+++ 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java
@@ -268,7 +268,7 @@ public class FeaturesSelectionStream extends TupleStream 
implements Expressible
   private List<String> getShardUrls() throws IOException {
     try {
       var cloudSolrClient = clientCache.getCloudSolrClient(zkHost);
-      Slice[] slices = CloudSolrStream.getSlices(this.collection, 
cloudSolrClient, false);
+      List<Slice> slices = CloudSolrStream.getSlices(this.collection, 
cloudSolrClient, false);
       Set<String> liveNodes = cloudSolrClient.getClusterState().getLiveNodes();
 
       List<String> baseUrls = new ArrayList<>();
diff --git 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java
 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java
index 2ac825c7ea4..bd6bce7c31b 100644
--- 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java
+++ 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java
@@ -374,7 +374,7 @@ public class TextLogitStream extends TupleStream implements 
Expressible {
   protected List<String> getShardUrls() throws IOException {
     try {
       var cloudSolrClient = clientCache.getCloudSolrClient(zkHost);
-      Slice[] slices = CloudSolrStream.getSlices(this.collection, 
cloudSolrClient, false);
+      List<Slice> slices = CloudSolrStream.getSlices(this.collection, 
cloudSolrClient, false);
 
       Set<String> liveNodes = cloudSolrClient.getClusterState().getLiveNodes();
 
diff --git 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
index b3bc4967c6d..0d2e4a46d1b 100644
--- 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
+++ 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
@@ -409,7 +409,7 @@ public class TopicStream extends CloudSolrStream implements 
Expressible {
   private void getCheckpoints() throws IOException {
     var cloudSolrClient = clientCache.getCloudSolrClient(zkHost);
     this.checkpoints = new HashMap<>();
-    Slice[] slices = CloudSolrStream.getSlices(this.collection, 
cloudSolrClient, false);
+    List<Slice> slices = CloudSolrStream.getSlices(this.collection, 
cloudSolrClient, false);
     Set<String> liveNodes = cloudSolrClient.getClusterState().getLiveNodes();
 
     for (Slice slice : slices) {
@@ -489,7 +489,7 @@ public class TopicStream extends CloudSolrStream implements 
Expressible {
 
   private void getPersistedCheckpoints() throws IOException {
     var cloudSolrClient = clientCache.getCloudSolrClient(zkHost);
-    Slice[] slices = CloudSolrStream.getSlices(checkpointCollection, 
cloudSolrClient, false);
+    List<Slice> slices = CloudSolrStream.getSlices(checkpointCollection, 
cloudSolrClient, false);
 
     Set<String> liveNodes = cloudSolrClient.getClusterState().getLiveNodes();
 
@@ -523,7 +523,7 @@ public class TopicStream extends CloudSolrStream implements 
Expressible {
   @Override
   protected void constructStreams() throws IOException {
     var cloudSolrClient = clientCache.getCloudSolrClient(zkHost);
-    Slice[] slices = CloudSolrStream.getSlices(this.collection, 
cloudSolrClient, false);
+    List<Slice> slices = CloudSolrStream.getSlices(this.collection, 
cloudSolrClient, false);
 
     ModifiableSolrParams mParams = new ModifiableSolrParams(params);
     mParams.set(DISTRIB, "false"); // We are the aggregator.
diff --git 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
index 435d5984083..78398713e74 100644
--- 
a/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
+++ 
b/solr/solrj-streaming/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
@@ -151,7 +151,7 @@ public abstract class TupleStream implements Closeable, 
Serializable, MapWriter
     try {
       CloudSolrClient cloudSolrClient = 
solrClientCache.getCloudSolrClient(zkHost);
       ClusterState clusterState = 
cloudSolrClient.getClusterStateProvider().getClusterState();
-      Slice[] slices = CloudSolrStream.getSlices(collection, cloudSolrClient, 
true);
+      List<Slice> slices = CloudSolrStream.getSlices(collection, 
cloudSolrClient, true);
       Set<String> liveNodes = clusterState.getLiveNodes();
 
       RequestReplicaListTransformerGenerator 
requestReplicaListTransformerGenerator;
diff --git 
a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java 
b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index d5a12688a7a..4922edc8aaa 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -494,7 +494,7 @@ public abstract class CloudSolrClient extends SolrClient {
   private Map<String, List<String>> buildUrlMap(
       DocCollection col, ReplicaListTransformer replicaListTransformer) {
     Map<String, List<String>> urlMap = new HashMap<>();
-    Slice[] slices = col.getActiveSlicesArr();
+    Collection<Slice> slices = col.getActiveSlices();
     Set<String> liveNodes = getClusterStateProvider().getLiveNodes();
     for (Slice slice : slices) {
       String name = slice.getName();
@@ -1218,7 +1218,7 @@ public abstract class CloudSolrClient extends SolrClient {
       NamedList<NamedList<?>> routes = ((RouteResponse<?>) 
resp).getRouteResponses();
       DocCollection coll = getDocCollection(collection, null);
       Map<String, String> leaders = new HashMap<>();
-      for (Slice slice : coll.getActiveSlicesArr()) {
+      for (Slice slice : coll.getActiveSlices()) {
         Replica leader = slice.getLeader();
         if (leader != null) {
           String leaderUrl = leader.getBaseUrl() + "/" + leader.getCoreName();
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java 
b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
index 2d977a9cf13..18e93ec7eb6 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
@@ -29,17 +29,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
-import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.stream.Stream;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.DocCollection.CollectionStateProps;
-import org.apache.solr.common.cloud.Replica.ReplicaStateProps;
 import org.apache.solr.common.util.CollectionUtil;
 import org.apache.solr.common.util.Utils;
 import org.noggit.JSONParser;
@@ -165,65 +162,11 @@ public class ClusterState implements MapWriter {
     return immutableCollectionStates.keySet();
   }
 
-  /**
-   * Get a map of collection name vs DocCollection objects
-   *
-   * <p>Implementation note: This method resolves the collection reference by 
calling {@link
-   * CollectionRef#get()} which can make a call to ZooKeeper. This is 
necessary because the
-   * semantics of how collection list is loaded have changed in SOLR-6629.
-   *
-   * @return a map of collection name vs DocCollection object
-   * @deprecated see {@link #collectionStream()}
-   */
-  @Deprecated
-  public Map<String, DocCollection> getCollectionsMap() {
-    Map<String, DocCollection> result = 
CollectionUtil.newHashMap(collectionStates.size());
-    for (Entry<String, CollectionRef> entry : collectionStates.entrySet()) {
-      DocCollection collection = entry.getValue().get();
-      if (collection != null) {
-        result.put(entry.getKey(), collection);
-      }
-    }
-    return result;
-  }
-
   /** Get names of the currently live nodes. */
   public Set<String> getLiveNodes() {
     return liveNodes;
   }
 
-  @Deprecated
-  public String getShardId(String nodeName, String coreName) {
-    return getShardId(null, nodeName, coreName);
-  }
-
-  @Deprecated
-  public String getShardId(String collectionName, String nodeName, String 
coreName) {
-    if (coreName == null || nodeName == null) {
-      return null;
-    }
-    Collection<CollectionRef> states = Collections.emptyList();
-    if (collectionName != null) {
-      CollectionRef c = collectionStates.get(collectionName);
-      if (c != null) states = Collections.singletonList(c);
-    } else {
-      states = collectionStates.values();
-    }
-
-    for (CollectionRef ref : states) {
-      DocCollection coll = ref.get();
-      if (coll == null) continue; // this collection got removed in between, 
skip
-      // TODO: for really large clusters, we could 'index' on this
-      return Optional.ofNullable(coll.getReplicasOnNode(nodeName)).stream()
-          .flatMap(List::stream)
-          .filter(r -> coreName.equals(r.getStr(ReplicaStateProps.CORE_NAME)))
-          .map(Replica::getShard)
-          .findAny()
-          .orElse(null);
-    }
-    return null;
-  }
-
   @Deprecated
   public Map<String, List<Replica>> getReplicaNamesPerCollectionOnNode(final 
String nodeName) {
     Map<String, List<Replica>> replicaNamesPerCollectionOnNode = new 
HashMap<>();
@@ -233,7 +176,7 @@ public class ClusterState implements MapWriter {
         .forEach(
             col -> {
               List<Replica> replicas = col.getReplicasOnNode(nodeName);
-              if (replicas != null && !replicas.isEmpty()) {
+              if (!replicas.isEmpty()) {
                 replicaNamesPerCollectionOnNode.put(col.getName(), replicas);
               }
             });
@@ -282,11 +225,6 @@ public class ClusterState implements MapWriter {
     return createFromCollectionMap(version, stateMap, liveNodes, creationTime, 
prsSupplier);
   }
 
-  @Deprecated
-  public static ClusterState createFromJson(int version, byte[] bytes, 
Set<String> liveNodes) {
-    return createFromJson(version, bytes, liveNodes, Instant.EPOCH, null);
-  }
-
   @Deprecated
   public static ClusterState createFromCollectionMap(
       int version,
@@ -311,12 +249,6 @@ public class ClusterState implements MapWriter {
     return new ClusterState(collections, liveNodes);
   }
 
-  @Deprecated
-  public static ClusterState createFromCollectionMap(
-      int version, Map<String, Object> stateMap, Set<String> liveNodes) {
-    return createFromCollectionMap(version, stateMap, liveNodes, 
Instant.EPOCH, null);
-  }
-
   /**
    * @lucene.internal
    */
@@ -398,17 +330,6 @@ public class ClusterState implements MapWriter {
     this.liveNodes = Set.copyOf(liveNodes);
   }
 
-  /**
-   * Be aware that this may return collections which may not exist now. You 
can confirm that this
-   * collection exists after verifying CollectionRef.get() != null
-   *
-   * @deprecated see {@link #collectionStream()}
-   */
-  @Deprecated
-  public Map<String, CollectionRef> getCollectionStates() {
-    return immutableCollectionStates;
-  }
-
   /**
    * Streams the resolved {@link DocCollection}s, which will often fetch from 
ZooKeeper for each one
    * for a many-collection scenario. Use this sparingly; some users have 
thousands of collections!
@@ -417,17 +338,6 @@ public class ClusterState implements MapWriter {
     return 
collectionStates.values().stream().map(CollectionRef::get).filter(Objects::nonNull);
   }
 
-  /**
-   * Calls {@code consumer} with a resolved {@link DocCollection}s for all 
collections. Use this
-   * sparingly in case there are many collections.
-   *
-   * @deprecated see {@link #collectionStream()}
-   */
-  @Deprecated
-  public void forEachCollection(Consumer<DocCollection> consumer) {
-    collectionStream().forEach(consumer);
-  }
-
   public static class CollectionRef {
     protected final AtomicInteger gets = new AtomicInteger();
     private final DocCollection coll;
diff --git 
a/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java 
b/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java
index 38ea73b0925..c13cc4fedef 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java
@@ -225,7 +225,7 @@ public class CompositeIdRouter extends HashBasedRouter {
     Range completeRange = new KeyParser(id).getRange();
 
     List<Slice> targetSlices = new ArrayList<>(1);
-    for (Slice slice : collection.getActiveSlicesArr()) {
+    for (Slice slice : collection.getActiveSlices()) {
       Range range = slice.getRange();
       if (range != null && range.overlaps(completeRange)) {
         targetSlices.add(slice);
diff --git 
a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java 
b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
index 234d8cec93d..3e112b60264 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
@@ -32,7 +32,6 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiConsumer;
-import java.util.function.BiPredicate;
 import java.util.function.Supplier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,7 +50,6 @@ public class DocCollection extends ZkNodeProps implements 
Iterable<Slice> {
   private final String configName;
   private final Map<String, Slice> slices;
   private final Map<String, Slice> activeSlices;
-  private final Slice[] activeSlicesArr;
   private final Map<String, List<Replica>> nodeNameReplicas;
   private final DocRouter router;
   private final String znode;
@@ -64,28 +62,6 @@ public class DocCollection extends ZkNodeProps implements 
Iterable<Slice> {
   private final Map<String, Replica> replicaMap = new HashMap<>();
   private AtomicReference<PerReplicaStates> perReplicaStatesRef;
 
-  /**
-   * @see DocCollection#create(String, Map, Map, DocRouter, int, Instant, 
PrsSupplier)
-   */
-  @Deprecated
-  public DocCollection(
-      String name, Map<String, Slice> slices, Map<String, Object> props, 
DocRouter router) {
-    this(name, slices, props, router, Integer.MAX_VALUE, Instant.EPOCH, null);
-  }
-
-  /**
-   * @see DocCollection#create(String, Map, Map, DocRouter, int, Instant, 
PrsSupplier)
-   */
-  @Deprecated
-  public DocCollection(
-      String name,
-      Map<String, Slice> slices,
-      Map<String, Object> props,
-      DocRouter router,
-      int zkVersion) {
-    this(name, slices, props, router, zkVersion, Instant.EPOCH, null);
-  }
-
   /**
    * @param name The name of the collection
    * @param slices The logical shards of the collection. This is used directly 
and a copy is not
@@ -146,7 +122,6 @@ public class DocCollection extends ZkNodeProps implements 
Iterable<Slice> {
         }
       }
     }
-    this.activeSlicesArr = activeSlices.values().toArray(new Slice[0]);
     this.router = router;
     this.znode = getCollectionPath(name);
     assert name != null && slices != null;
@@ -183,8 +158,8 @@ public class DocCollection extends ZkNodeProps implements 
Iterable<Slice> {
             CollectionStateProps.PER_REPLICA_STATE + " = true , but 
prsSupplier is not provided");
       }
 
-      if (!hasAnyReplica(
-          slices)) { // a special case, if there is no replica, it should not 
fetch (first PRS
+      if (!hasAnyReplica(slices)) {
+        // a special case, if there is no replica, it should not fetch (first 
PRS
         // collection creation with no replicas). Otherwise, it would trigger 
exception
         // on fetching a state.json that does not exist yet
         perReplicaStates = PerReplicaStates.empty(name);
@@ -329,12 +304,6 @@ public class DocCollection extends ZkNodeProps implements 
Iterable<Slice> {
     return activeSlices.values();
   }
 
-  /** Return array of active slices for this collection (performance 
optimization). */
-  @Deprecated
-  public Slice[] getActiveSlicesArr() {
-    return activeSlicesArr;
-  }
-
   /** Get the map of all slices (sliceName-&gt;Slice) for this collection. */
   public Map<String, Slice> getSlicesMap() {
     return slices;
@@ -345,15 +314,9 @@ public class DocCollection extends ZkNodeProps implements 
Iterable<Slice> {
     return activeSlices;
   }
 
-  /** Get the list of replicas hosted on the given node or <code>null</code> 
if none. */
-  @Deprecated // see getReplicasOnNode
-  public List<Replica> getReplicas(String nodeName) {
-    return getReplicasOnNode(nodeName);
-  }
-
-  /** Get the list of replicas hosted on the given node or <code>null</code> 
if none. */
+  /** Get the list of replicas hosted on the given node, or an empty list if 
none. */
   public List<Replica> getReplicasOnNode(String nodeName) {
-    return nodeNameReplicas.get(nodeName);
+    return nodeNameReplicas.getOrDefault(nodeName, List.of());
   }
 
   public int getZNodeVersion() {
@@ -474,23 +437,6 @@ public class DocCollection extends ZkNodeProps implements 
Iterable<Slice> {
     return replicas;
   }
 
-  /**
-   * @param predicate test against shardName vs. replica
-   * @return the first replica that matches the predicate
-   */
-  @Deprecated // just one test; move it
-  public Replica getReplica(BiPredicate<String, Replica> predicate) {
-    final Replica[] result = new Replica[1];
-    forEachReplica(
-        (s, replica) -> {
-          if (result[0] != null) return;
-          if (predicate.test(s, replica)) {
-            result[0] = replica;
-          }
-        });
-    return result[0];
-  }
-
   @Deprecated // just tests, so move out or make package-protected
   public List<Replica> getReplicas(EnumSet<Replica.Type> s) {
     List<Replica> replicas = new ArrayList<>();
@@ -500,18 +446,6 @@ public class DocCollection extends ZkNodeProps implements 
Iterable<Slice> {
     return replicas;
   }
 
-  /** Get the shardId of a core on a specific node */
-  @Deprecated // only one usage; obscure looking
-  public String getShardId(String nodeName, String coreName) {
-    for (Slice slice : this) {
-      for (Replica replica : slice) {
-        if (Objects.equals(replica.getNodeName(), nodeName)
-            && Objects.equals(replica.getCoreName(), coreName)) return 
slice.getName();
-      }
-    }
-    return null;
-  }
-
   @Override
   public boolean equals(Object that) {
     if (!(that instanceof DocCollection other)) return false;
@@ -526,33 +460,6 @@ public class DocCollection extends ZkNodeProps implements 
Iterable<Slice> {
     return Objects.hash(name, znodeVersion, getChildNodesVersion());
   }
 
-  /**
-   * @return the number of replicas of type {@link 
org.apache.solr.common.cloud.Replica.Type#NRT}
-   *     this collection was created with
-   */
-  @Deprecated
-  public Integer getNumNrtReplicas() {
-    return getNumReplicas(Replica.Type.NRT);
-  }
-
-  /**
-   * @return the number of replicas of type {@link 
org.apache.solr.common.cloud.Replica.Type#TLOG}
-   *     this collection was created with
-   */
-  @Deprecated
-  public Integer getNumTlogReplicas() {
-    return getNumReplicas(Replica.Type.TLOG);
-  }
-
-  /**
-   * @return the number of replicas of type {@link 
org.apache.solr.common.cloud.Replica.Type#PULL}
-   *     this collection was created with
-   */
-  @Deprecated
-  public Integer getNumPullReplicas() {
-    return getNumReplicas(Replica.Type.PULL);
-  }
-
   /**
    * @return the number of replicas of a given type this collection was 
created with
    */
@@ -568,12 +475,6 @@ public class DocCollection extends ZkNodeProps implements 
Iterable<Slice> {
     return perReplicaStatesRef != null ? perReplicaStatesRef.get() : null;
   }
 
-  @Deprecated
-  public int getExpectedReplicaCount(Replica.Type type, int def) {
-    // def is kept for backwards compatibility.
-    return numReplicas.get(type);
-  }
-
   /** JSON properties related to a collection's state. */
   public interface CollectionStateProps {
     String REPLICATION_FACTOR = "replicationFactor";
diff --git 
a/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java 
b/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java
index 3e3af9aefb2..0a02c39d8e5 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java
@@ -69,7 +69,7 @@ public abstract class HashBasedRouter extends DocRouter {
   }
 
   protected Slice hashToSlice(int hash, DocCollection collection) {
-    final Slice[] slices = collection.getActiveSlicesArr();
+    final Collection<Slice> slices = collection.getActiveSlices();
     for (Slice slice : slices) {
       Range range = slice.getRange();
       if (range != null && range.includes(hash)) return slice;
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java 
b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
index e5102b0d3f1..85f6f43d2c0 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
@@ -168,12 +168,10 @@ public class Replica extends ZkNodeProps implements 
MapWriter {
   public final String core;
   public final Type type;
   public final String shard, collection;
+  private final State state;
   private String baseUrl, coreUrl; // Derived values
   private AtomicReference<PerReplicaStates> perReplicaStatesRef;
 
-  // mutable
-  private State state;
-
   void setPerReplicaStatesRef(AtomicReference<PerReplicaStates> 
perReplicaStatesRef) {
     this.perReplicaStatesRef = perReplicaStatesRef;
   }
@@ -238,9 +236,7 @@ public class Replica extends ZkNodeProps implements 
MapWriter {
     this.propMap.putAll(details);
     type =
         
Replica.Type.valueOf(String.valueOf(propMap.getOrDefault(ReplicaStateProps.TYPE,
 "NRT")));
-    if (state == null)
-      state =
-          
State.getState(String.valueOf(propMap.getOrDefault(ReplicaStateProps.STATE, 
"active")));
+    state = 
State.getState(String.valueOf(propMap.getOrDefault(ReplicaStateProps.STATE, 
"active")));
     validate();
   }
 
@@ -331,12 +327,6 @@ public class Replica extends ZkNodeProps implements 
MapWriter {
     return state;
   }
 
-  @Deprecated
-  public void setState(State state) {
-    this.state = state;
-    propMap.put(ReplicaStateProps.STATE, this.state.toString());
-  }
-
   public boolean isActive(Set<String> liveNodes) {
     return this.node != null && liveNodes.contains(this.node) && getState() == 
State.ACTIVE;
   }
@@ -388,9 +378,9 @@ public class Replica extends ZkNodeProps implements 
MapWriter {
   }
 
   public Replica copyWith(State state) {
-    Replica r = new Replica(name, propMap, collection, shard);
-    r.setState(state);
-    return r;
+    Map<String, Object> newProps = new LinkedHashMap<>(propMap);
+    newProps.put(ReplicaStateProps.STATE, state.toString());
+    return new Replica(name, newProps, collection, shard);
   }
 
   public PerReplicaStates.State getReplicaState() {
diff --git 
a/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java
 
b/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java
index 81b210c01a0..e0129faaf10 100644
--- 
a/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java
+++ 
b/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java
@@ -95,7 +95,7 @@ public class PerReplicaStatesIntegrationTest extends 
SolrCloudTestCase {
       assertEquals(5, prs.states.size());
 
       // Test delete replica
-      Replica leader = c.getReplica((s, replica) -> replica.isLeader());
+      Replica leader = 
c.getReplicas().stream().filter(Replica::isLeader).findFirst().orElseThrow();
       CollectionAdminRequest.deleteReplica(testCollection, leader.shard, 
leader.getName())
           .process(cluster.getSolrClient());
       cluster.waitForActiveCollection(testCollection, 2, 4);
@@ -376,7 +376,7 @@ public class PerReplicaStatesIntegrationTest extends 
SolrCloudTestCase {
       // +1 for a new replica
       assertEquals(4, stat.getVersion());
       DocCollection c = cluster.getZkStateReader().getCollection(PRS_COLL);
-      Replica newreplica = c.getReplica((s, replica) -> 
replica.node.equals(j2.getNodeName()));
+      Replica newreplica = c.getReplicasOnNode(j2.getNodeName()).getFirst();
 
       // let's stop the old leader
       JettySolrRunner oldJetty = cluster.getReplicaJetty(leader);
diff --git 
a/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
 
b/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
index cb622bb42e8..87377bb02e5 100644
--- 
a/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
+++ 
b/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
@@ -44,6 +44,7 @@ import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ImplicitDocRouter;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
@@ -449,15 +450,15 @@ public abstract class AbstractCloudBackupRestoreTestCase 
extends SolrCloudTestCa
     assertEquals(
         restoreCollection.toString(),
         restoreReplcationFactor,
-        restoreCollection.getNumNrtReplicas().intValue());
+        restoreCollection.getNumReplicas(Replica.Type.NRT));
     assertEquals(
         restoreCollection.toString(),
         restorePullReplicas,
-        restoreCollection.getNumPullReplicas().intValue());
+        restoreCollection.getNumReplicas(Replica.Type.PULL));
     assertEquals(
         restoreCollection.toString(),
         restoreTlogReplicas,
-        restoreCollection.getNumTlogReplicas().intValue());
+        restoreCollection.getNumReplicas(Replica.Type.TLOG));
 
     // SOLR-12605: Add more docs after restore is complete to see if they are 
getting added fine
     // explicitly querying the leaders. If we use CloudSolrClient there is no 
guarantee that we'll

Reply via email to