You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2020/11/14 19:53:02 UTC

[lucene-solr] branch reference_impl_dev updated (36b6375 -> 10468b8)

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

markrmiller pushed a change to branch reference_impl_dev
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


    from 36b6375  @1200 Test Hardening.
     new 09f5d2b  @1201 Pull this out.
     new 10468b8  @1202 Remove the rest of update lock.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../solr/handler/admin/MetricsHistoryHandler.java  |   4 +-
 .../org/apache/solr/schema/ManagedIndexSchema.java |   4 +-
 .../solr/search/join/ScoreJoinQParserPlugin.java   |   2 +-
 .../java/org/apache/solr/servlet/HttpSolrCall.java |   4 +-
 .../processor/DistributedZkUpdateProcessor.java    |   6 +-
 .../DocExpirationUpdateProcessorFactory.java       |   4 +-
 .../processor/RoutedAliasUpdateProcessor.java      |   7 +-
 .../solr/cloud/TestDownShardTolerantSearch.java    |  42 +++----
 .../client/solrj/impl/BaseCloudSolrClient.java     |   4 +-
 .../client/solrj/io/stream/CloudSolrStream.java    |   5 +-
 .../client/solrj/io/stream/DeepRandomStream.java   |   5 +-
 .../apache/solr/common/cloud/ClusterStateUtil.java |   4 +-
 .../apache/solr/common/cloud/DocCollection.java    |  14 +--
 .../apache/solr/common/cloud/HashBasedRouter.java  |   2 +-
 .../apache/solr/common/cloud/ZkStateReader.java    | 125 ++++++---------------
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |   2 +-
 16 files changed, 91 insertions(+), 143 deletions(-)


[lucene-solr] 02/02: @1202 Remove the rest of update lock.

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 10468b8400405807cb05916d75bdb2a99a9e5101
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Sat Nov 14 13:52:12 2020 -0600

    @1202 Remove the rest of update lock.
---
 .../solr/cloud/TestDownShardTolerantSearch.java    |  42 +++----
 .../apache/solr/common/cloud/ZkStateReader.java    | 125 ++++++---------------
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |   2 +-
 3 files changed, 60 insertions(+), 109 deletions(-)

diff --git a/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java b/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
index 8b04334..d9f5745 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
@@ -18,6 +18,7 @@ package org.apache.solr.cloud;
 
 import java.lang.invoke.MethodHandles;
 
+import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -46,9 +47,7 @@ public class TestDownShardTolerantSearch extends SolrCloudTestCase {
 
   @BeforeClass
   public static void setupCluster() throws Exception {
-    configureCluster(2)
-        .addConfig("conf", configset("cloud-minimal"))
-        .configure();
+    configureCluster(2).addConfig("conf", configset("cloud-minimal")).configure();
   }
 
   @Test
@@ -68,26 +67,29 @@ public class TestDownShardTolerantSearch extends SolrCloudTestCase {
 
     JettySolrRunner stoppedServer = cluster.stopJettySolrRunner(0);
 
-    while (true) {
-      try {
-        response = cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, true));
-        break;
-      } catch (BaseHttpSolrClient.RemoteExecutionException e) {
-        // a remote node we are proxied too may still think this is live, try again
-        if (!e.getMessage().contains("Connection refused")) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+    try (SolrClient client = cluster.buildSolrClient()) {
+      for (int i = 0; i < 10; i++) {
+
+        try {
+          response = client.query("tolerant", new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, true));
+          break;
+        } catch (BaseHttpSolrClient.RemoteExecutionException e) {
+          // a remote node we are proxied too may still think this is live, try again
+          if (!e.getMessage().contains("Connection refused")) {
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+          }
         }
       }
-    }
 
-    assertThat(response.getStatus(), is(0));
-    assertTrue(response.getResults().getNumFound() > 0);
+      assertThat(response.getStatus(), is(0));
+      assertTrue(response.getResults().getNumFound() > 0);
 
-    SolrServerException e = expectThrows(SolrServerException.class, "Request should have failed because we killed shard1 jetty",
-        () -> cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, false)));
-    assertNotNull(e.getCause());
-    if (!e.getCause().getMessage().contains("Connection refused")) {
-      assertTrue("Error message from server should have the name of the down shard " + e.getCause().getMessage(), e.getCause().getMessage().contains("shard"));
+      SolrServerException e = expectThrows(SolrServerException.class, "Request should have failed because we killed shard1 jetty",
+          () -> cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, false)));
+      assertNotNull(e.getCause());
+      if (!e.getCause().getMessage().contains("Connection refused")) {
+        assertTrue("Error message from server should have the name of the down shard " + e.getCause().getMessage(), e.getCause().getMessage().contains("shard"));
+      }
     }
   }
-  }
+}
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 37e652c..2c7341b 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -175,8 +175,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
   public static final String SHARD_LEADERS_ZKNODE = "leaders";
   public static final String ELECTION_NODE = "election";
 
-  private final ReentrantLock updateLock = new ReentrantLock(true);
-
   /**
    * "Interesting" and actively watched Collections.
    */
@@ -347,8 +345,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
   // don't call this, used in one place
 
   public void forciblyRefreshAllClusterStateSlow() {
-    updateLock.lock();
-
     // No need to set watchers because we should already have watchers registered for everything.
     try {
       refreshCollectionList(null);
@@ -369,8 +365,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     } catch (InterruptedException e) {
       ParWork.propagateInterrupt(e);
       throw new SolrException(ErrorCode.SERVER_ERROR, e);
-    } finally {
-      updateLock.unlock();
     }
 
   }
@@ -393,12 +387,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
       if (nu == null) return -1;
       if (nu.getZNodeVersion() > collection.getZNodeVersion()) {
         if (updateWatchedCollection(coll, nu)) {
-     //     updateLock.lock();
-          try {
-            constructState(Collections.singleton(coll));
-          } finally {
-       //     updateLock.unlock();
-          }
+          constructState(Collections.singleton(coll));
         }
         collection = nu;
       }
@@ -520,28 +509,25 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
               return;
             }
             try {
-              updateLock.lock();
+
+              log.info("Updating [{}] ... ", SOLR_SECURITY_CONF_PATH);
+
+              // remake watch
+              final Stat stat = new Stat();
+              byte[] data = "{}".getBytes(StandardCharsets.UTF_8);
+              if (EventType.NodeDeleted.equals(event.getType())) {
+                // Node deleted, just recreate watch without attempting a read - SOLR-9679
+                getZkClient().exists(SOLR_SECURITY_CONF_PATH, this);
+              } else {
+                data = getZkClient().getData(SOLR_SECURITY_CONF_PATH, this, stat);
+              }
               try {
-                log.info("Updating [{}] ... ", SOLR_SECURITY_CONF_PATH);
-
-                // remake watch
-                final Stat stat = new Stat();
-                byte[] data = "{}".getBytes(StandardCharsets.UTF_8);
-                if (EventType.NodeDeleted.equals(event.getType())) {
-                  // Node deleted, just recreate watch without attempting a read - SOLR-9679
-                  getZkClient().exists(SOLR_SECURITY_CONF_PATH, this);
-                } else {
-                  data = getZkClient().getData(SOLR_SECURITY_CONF_PATH, this, stat);
-                }
-                try {
-                  callback.call(new Pair<>(data, stat));
-                } catch (Exception e) {
-                  log.error("Error running collections node listener", e);
-                  return;
-                }
-              } finally {
-                updateLock.unlock();
+                callback.call(new Pair<>(data, stat));
+              } catch (Exception e) {
+                log.error("Error running collections node listener", e);
+                return;
               }
+
             } catch (KeeperException e) {
               log.error("A ZK error has occurred", e);
               return;
@@ -555,7 +541,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
   /**
    * Construct the total state view from all sources.
-   * Must hold {@link #getUpdateLock()} before calling this.
    *
    * @param changedCollections collections that have changed since the last call,
    *                           and that should fire notifications
@@ -756,21 +741,18 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     // Can't lock getUpdateLock() until we release the other, it would cause deadlock.
     SortedSet<String> oldLiveNodes;
     SortedSet<String> newLiveNodes = null;
-    updateLock.lock();
-    try {
-      try {
-        List<String> nodeList = zkClient.getChildren(LIVE_NODES_ZKNODE, watcher, true);
-        newLiveNodes = new TreeSet<>(nodeList);
-      } catch (KeeperException.NoNodeException e) {
-        newLiveNodes = emptySortedSet();
-      }
 
-      oldLiveNodes = this.liveNodes;
-      this.liveNodes = newLiveNodes;
-      clusterState.setLiveNodes(newLiveNodes);
-    } finally {
-      updateLock.unlock();
+    try {
+      List<String> nodeList = zkClient.getChildren(LIVE_NODES_ZKNODE, watcher, true);
+      newLiveNodes = new TreeSet<>(nodeList);
+    } catch (KeeperException.NoNodeException e) {
+      newLiveNodes = emptySortedSet();
     }
+
+    oldLiveNodes = this.liveNodes;
+    this.liveNodes = newLiveNodes;
+    clusterState.setLiveNodes(newLiveNodes);
+
     if (oldLiveNodes.size() != newLiveNodes.size()) {
       if (log.isInfoEnabled()) {
         log.info("Updated live nodes from ZooKeeper... ({}) -> ({})", oldLiveNodes.size(), newLiveNodes.size());
@@ -828,10 +810,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     return liveNodes;
   }
 
-  public ReentrantLock getUpdateLock() {
-    return updateLock;
-  }
-
   public void close() {
     log.info("Closing ZkStateReader");
     assert closeTracker.close();
@@ -1323,12 +1301,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
         }
         DocCollection newState = fetchCollectionState(coll, this);
         updateWatchedCollection(coll, newState);
-      //  updateLock.lock();
-        try {
-          constructState(Collections.singleton(coll));
-        } finally {
-     //     updateLock.unlock();
-        }
+
+        constructState(Collections.singleton(coll));
 
       } catch (KeeperException e) {
         log.error("Unwatched collection: [{}]", coll, e);
@@ -1348,7 +1322,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
         public void process(WatchedEvent event) {
           log.info("_statupdates event {}", event);
           try {
-            updateLock.lock();
 
 //            if (event.getType() == EventType.NodeDataChanged ||
 //                event.getType() == EventType.NodeDeleted || event.getType() == EventType.NodeCreated) {
@@ -1371,8 +1344,6 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
             log.error("Unwatched collection: [{}]", coll, e);
           } catch (InterruptedException e) {
             log.error("Unwatched collection: [{}]", coll, e);
-          } finally {
-            updateLock.unlock();
           }
 
         }
@@ -1664,17 +1635,10 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
         log.error("An error has occurred", e);
         return;
       }
-     // updateLock.lock();
-      try {
-        constructState(Collections.emptySet());
-      } finally {
-    //    updateLock.unlock();
-      }
-    }
 
-    /**
-     * Must hold {@link #getUpdateLock()} before calling this method.
-     */
+      constructState(Collections.emptySet());
+    }
+    
     public void refreshAndWatch() {
       try {
         refreshCollectionList(this);
@@ -1840,12 +1804,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     });
 
     if (reconstructState.get()) {
-    //  updateLock.lock();
-      try {
-        constructState(Collections.emptySet());
-      } finally {
-      //  updateLock.unlock();
-      }
+      constructState(Collections.emptySet());
     }
   }
 
@@ -2111,12 +2070,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
     }
 
     if (reconstructState.get()) {
-     // updateLock.lock();
-      try {
-        constructState(Collections.emptySet());
-      } finally {
-     //   updateLock.unlock();
-      }
+      constructState(Collections.emptySet());
     }
   }
 
@@ -2305,13 +2259,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
 
   // called by createClusterStateWatchersAndUpdate()
   private void refreshAliases(AliasesManager watcher) throws KeeperException, InterruptedException {
-   // updateLock.lock();
-    try {
-      constructState(Collections.emptySet());
-      zkClient.exists(ALIASES, watcher);
-    } finally {
-    //  updateLock.unlock();
-    }
+    constructState(Collections.emptySet());
+    zkClient.exists(ALIASES, watcher);
     aliasesManager.update();
   }
 
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
index 5cd5123..21bc927 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
@@ -743,7 +743,7 @@ public class MiniSolrCloudCluster {
     return solrZkClient;
   }
   
-  protected CloudHttp2SolrClient buildSolrClient() {
+  public CloudHttp2SolrClient buildSolrClient() {
    // return new CloudHttp2SolrClient.Builder(Collections.singletonList(zkServer.getZkHost()), Optional.of("/solr")).build();
     zkStateReader = new ZkStateReader(zkServer.getZkAddress(), 15000, 30000);
     zkStateReader.createClusterStateWatchersAndUpdate();


[lucene-solr] 01/02: @1201 Pull this out.

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 09f5d2b2801c2dcc30dd276ba2c6d5507057e11e
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Sat Nov 14 13:20:33 2020 -0600

    @1201 Pull this out.
---
 .../apache/solr/handler/admin/MetricsHistoryHandler.java   |  4 ++--
 .../java/org/apache/solr/schema/ManagedIndexSchema.java    |  4 ++--
 .../apache/solr/search/join/ScoreJoinQParserPlugin.java    |  2 +-
 .../src/java/org/apache/solr/servlet/HttpSolrCall.java     |  4 ++--
 .../update/processor/DistributedZkUpdateProcessor.java     |  6 +++---
 .../processor/DocExpirationUpdateProcessorFactory.java     |  4 ++--
 .../solr/update/processor/RoutedAliasUpdateProcessor.java  |  7 ++++---
 .../apache/solr/client/solrj/impl/BaseCloudSolrClient.java |  4 ++--
 .../solr/client/solrj/io/stream/CloudSolrStream.java       |  5 +++--
 .../solr/client/solrj/io/stream/DeepRandomStream.java      |  5 +++--
 .../org/apache/solr/common/cloud/ClusterStateUtil.java     |  4 ++--
 .../java/org/apache/solr/common/cloud/DocCollection.java   | 14 ++++----------
 .../java/org/apache/solr/common/cloud/HashBasedRouter.java |  2 +-
 13 files changed, 31 insertions(+), 34 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java
index 8571ed3..601ba6e 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java
@@ -504,8 +504,8 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss
         Map<String, Number> perReg = totals
             .computeIfAbsent(Group.collection, g -> new HashMap<>())
             .computeIfAbsent(registry, r -> new HashMap<>());
-        Slice[] slices = coll.getActiveSlicesArr();
-        perReg.put(NUM_SHARDS_KEY, slices.length);
+        Collection<Slice> slices = coll.getActiveSlices();
+        perReg.put(NUM_SHARDS_KEY, slices.size());
         DoubleAdder numActiveReplicas = new DoubleAdder();
         for (Slice s : slices) {
           s.forEach(r -> {
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 bf60922..3255baf 100644
--- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
@@ -326,8 +326,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();
+    if (docCollection != null && docCollection .getActiveSlices().size() > 0) {
+      Collection<Slice> activeSlices = docCollection.getActiveSlices();
       for (Slice next : activeSlices) {
         Map<String, Replica> replicasMap = next.getReplicasMap();
         if (replicasMap != null) {
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 7bd78c0..064faee 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
@@ -306,7 +306,7 @@ public class ScoreJoinQParserPlugin extends QParserPlugin {
     String fromReplica = null;
 
     String nodeName = zkController.getNodeName();
-    for (Slice slice : zkController.getClusterState().getCollection(fromIndex).getActiveSlicesArr()) {
+    for (Slice slice : zkController.getClusterState().getCollection(fromIndex).getActiveSlices()) {
       if (fromReplica != null)
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
             "SolrCloud join: multiple shards not yet supported " + fromIndex);
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 3ae85e4..cfa4d7b 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -1089,8 +1089,8 @@ public class HttpSolrCall {
         if (entry.getValue() == null) {
           continue;
         }
-        final Slice[] activeCollectionSlices = entry.getValue().getActiveSlicesArr();
-        slices.addAll(Arrays.asList(activeCollectionSlices));
+        Collection<Slice> activeCollectionSlices = entry.getValue().getActiveSlices();
+        slices.addAll(activeCollectionSlices);
       }
     } else {
       for (Map.Entry<String, DocCollection> entry : clusterState.getCollectionsMap().entrySet()) {
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 47b7a5f..22a941b 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
@@ -977,9 +977,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, true);
-            if (docCollection != null && docCollection.getActiveSlicesArr().length > 0) {
-              final Slice[] activeSlices = docCollection.getActiveSlicesArr();
-              Slice any = activeSlices[0];
+            if (docCollection != null && docCollection.getActiveSlices().size() > 0) {
+              Collection<Slice> activeSlices = docCollection.getActiveSlices();
+              Slice any = activeSlices.iterator().next();
               if (nodes == null) nodes = new ArrayList<>();
               nodes.add(new SolrCmdDistributor.StdNode(zkController.getZkStateReader(), 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 09f25f2..38fefe7 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
@@ -476,11 +476,11 @@ public final class DocExpirationUpdateProcessorFactory
     String col = desc.getCollectionName();
 
     DocCollection docCollection = zk.getClusterState().getCollection(col);
-    if (docCollection.getActiveSlicesArr().length == 0) {
+    if (docCollection.getActiveSlices().size() == 0) {
       log.error("Collection {} has no active Slices?", col);
       return false;
     }
-    List<Slice> slices = new ArrayList<>(Arrays.asList(docCollection.getActiveSlicesArr()));
+    List<Slice> slices = new ArrayList<>(docCollection.getActiveSlices());
     Collections.sort(slices, COMPARE_SLICES_BY_NAME);
     Replica firstSliceLeader = slices.get(0).getLeader();
     if (null == firstSliceLeader) {
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 1b30e6f..cbbb14b 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
@@ -19,6 +19,7 @@ package org.apache.solr.update.processor;
 
 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;
@@ -252,11 +253,11 @@ public class RoutedAliasUpdateProcessor extends UpdateRequestProcessor {
   }
 
   private SolrCmdDistributor.Node lookupShardLeaderOfCollection(String collection) {
-    final Slice[] activeSlices = zkController.getClusterState().getCollection(collection).getActiveSlicesArr();
-    if (activeSlices.length == 0) {
+    Collection<Slice> activeSlices = zkController.getClusterState().getCollection(collection).getActiveSlices();
+    if (activeSlices.size() == 0) {
       throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Cannot route to collection " + collection);
     }
-    final Slice slice = activeSlices[0];
+    final Slice slice = activeSlices.iterator().next();
     return getLeaderNode(collection, slice);
   }
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java
index f4336e1..f32a81b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java
@@ -635,7 +635,7 @@ public abstract class BaseCloudSolrClient extends SolrClient {
 
   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();
     for (Slice slice : slices) {
       String name = slice.getName();
       List<Replica> sortedReplicas = new ArrayList<>();
@@ -1350,7 +1350,7 @@ public abstract class BaseCloudSolrClient extends SolrClient {
       NamedList routes = ((RouteResponse)resp).getRouteResponses();
       DocCollection coll = getDocCollection(collection, null);
       Map<String,String> leaders = new HashMap<String,String>();
-      for (Slice slice : coll.getActiveSlicesArr()) {
+      for (Slice slice : coll.getActiveSlices()) {
         Replica leader = slice.getLeader();
         if (leader != null) {
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
index 1c56f55..af7fc92 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
@@ -70,6 +70,7 @@ import static org.apache.solr.common.params.CommonParams.SORT;
 public class CloudSolrStream extends TupleStream implements Expressible {
 
   private static final long serialVersionUID = 1;
+  public static final Slice[] SLICES = new Slice[0];
 
   protected String zkHost;
   protected String collection;
@@ -352,7 +353,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
     List<Slice> slices = allCollections.stream()
         .map(collectionsMap::get)
         .filter(Objects::nonNull)
-        .flatMap(docCol -> Arrays.stream(docCol.getActiveSlicesArr()))
+        .flatMap(docCol -> docCol.getActiveSlices().stream())
         .collect(Collectors.toList());
     if (!slices.isEmpty()) {
       return slices.toArray(new Slice[slices.size()]);
@@ -361,7 +362,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
     // Check collection case insensitive
     for(Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
       if(entry.getKey().equalsIgnoreCase(collectionName)) {
-        return entry.getValue().getActiveSlicesArr();
+        return entry.getValue().getActiveSlices().toArray(SLICES);
       }
     }
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DeepRandomStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DeepRandomStream.java
index db3db4e..05acf97 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DeepRandomStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DeepRandomStream.java
@@ -67,6 +67,7 @@ import static org.apache.solr.common.params.CommonParams.SORT;
 public class DeepRandomStream extends TupleStream implements Expressible {
 
   private static final long serialVersionUID = 1;
+  public static final Slice[] SLICES = new Slice[0];
 
   protected String zkHost;
   protected String collection;
@@ -290,7 +291,7 @@ public class DeepRandomStream extends TupleStream implements Expressible {
     List<Slice> slices = allCollections.stream()
         .map(collectionsMap::get)
         .filter(Objects::nonNull)
-        .flatMap(docCol -> Arrays.stream(docCol.getActiveSlicesArr()))
+        .flatMap(docCol -> docCol.getActiveSlices().stream())
         .collect(Collectors.toList());
     if (!slices.isEmpty()) {
       return slices.toArray(new Slice[slices.size()]);
@@ -299,7 +300,7 @@ public class DeepRandomStream extends TupleStream implements Expressible {
     // Check collection case insensitive
     for(Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
       if(entry.getKey().equalsIgnoreCase(collectionName)) {
-        return entry.getValue().getActiveSlicesArr();
+        return entry.getValue().getActiveSlices().toArray(SLICES);
       }
     }
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
index 5dccb44..1d702bf 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
@@ -157,8 +157,8 @@ public class ClusterStateUtil {
   }
   
   public static int getLiveAndActiveReplicaCount(ZkStateReader zkStateReader, String collection) {
-    Slice[] slices;
-    slices = zkStateReader.getClusterState().getCollection(collection).getActiveSlicesArr();
+    Collection<Slice> slices;
+    slices = zkStateReader.getClusterState().getCollection(collection).getActiveSlices();
     int liveAndActive = 0;
     for (Slice slice : slices) {
       for (Replica replica : slice.getReplicas()) {
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 4dfdfb5..79a7e2b 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
@@ -18,6 +18,7 @@ package org.apache.solr.common.cloud;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -54,7 +55,6 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
   private final String name;
   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 Map<String, List<Replica>> nodeNameLeaderReplicas;
   private final DocRouter router;
@@ -104,7 +104,6 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
         addNodeNameReplica(replica);
       }
     }
-    this.activeSlicesArr = activeSlices.values().toArray(new Slice[activeSlices.size()]);
     this.router = router;
     assert name != null && slices != null;
   }
@@ -193,14 +192,9 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
    * Return the list of active slices for this collection.
    */
   public Collection<Slice> getActiveSlices() {
-    return activeSlices.values();
-  }
-
-  /**
-   * Return array of active slices for this collection (performance optimization).
-   */
-  public Slice[] getActiveSlicesArr() {
-    return activeSlicesArr;
+    List<Slice> slices = new ArrayList<>(activeSlices.values());
+    Collections.shuffle(slices);
+    return slices;
   }
 
   /**
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 4ac4e51..89d581b 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
@@ -59,7 +59,7 @@ public abstract class HashBasedRouter extends DocRouter {
   }
 
   protected Slice hashToSlice(int hash, DocCollection collection) {
-    final Slice[] slices = collection.getActiveSlicesArr();
+    Collection<Slice> slices = collection.getActiveSlices();
     for (Slice slice : slices) {
       Range range = slice.getRange();
       if (range != null && range.includes(hash)) return slice;