You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ep...@apache.org on 2023/03/27 16:26:24 UTC

[solr] branch branch_9x updated: SOLR-10466: setDefaultCollection should be deprecated in favor of SolrClientBuilder methods (#1256)

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

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


The following commit(s) were added to refs/heads/branch_9x by this push:
     new e725770d25e SOLR-10466: setDefaultCollection should be deprecated in favor of SolrClientBuilder methods (#1256)
e725770d25e is described below

commit e725770d25e70b6d9703fb4457fd5ffef9f2ed24
Author: Eric Pugh <ep...@opensourceconnections.com>
AuthorDate: Mon Mar 27 06:44:00 2023 -0400

    SOLR-10466: setDefaultCollection should be deprecated in favor of SolrClientBuilder methods (#1256)
    
    * Migrate setDefaultCollection method from CloudSolrClient to the appropriate Builder classes.
    
    * Eradicate the use of CloudSolrClient.setDefaultCollection from all the unit tests, making sure the logic of the tests is still the same.  This required introducing a cache of collection specific CloudSolrClients.
    
    ---------
    
    Co-authored-by: David Smiley <ds...@apache.org>
---
 solr/CHANGES.txt                                   |   6 +-
 .../client/solrj/impl/ConnectionReuseTest.java     |  26 +-
 .../apache/solr/cloud/AliasIntegrationTest.java    |  11 +-
 ...aosMonkeyNothingIsSafeWithPullReplicasTest.java |   6 +-
 .../org/apache/solr/cloud/CleanupOldIndexTest.java |   5 +-
 .../apache/solr/cloud/CollectionsAPISolrJTest.java |   7 +-
 .../apache/solr/cloud/DocValuesNotIndexedTest.java |   8 +-
 .../org/apache/solr/cloud/ForceLeaderTest.java     | 100 +++---
 .../solr/cloud/FullSolrCloudDistribCmdsTest.java   | 267 ++++++++++------
 .../solr/cloud/HttpPartitionOnCommitTest.java      |   2 -
 .../org/apache/solr/cloud/HttpPartitionTest.java   |  53 ++--
 .../cloud/LeaderFailoverAfterPartitionTest.java    |  16 +-
 .../apache/solr/cloud/LeaderTragicEventTest.java   |   5 +-
 .../org/apache/solr/cloud/MigrateRouteKeyTest.java |  27 +-
 .../solr/cloud/MissingSegmentRecoveryTest.java     |   5 +-
 .../solr/cloud/NestedShardedAtomicUpdateTest.java  |   3 +-
 .../solr/cloud/ParallelCommitExecutionTest.java    |  17 +-
 .../apache/solr/cloud/ReplicationFactorTest.java   | 108 +++----
 .../solr/cloud/SegmentTerminateEarlyTestState.java |  37 ++-
 .../apache/solr/cloud/SolrCloudExampleTest.java    |   8 +-
 .../test/org/apache/solr/cloud/SplitShardTest.java |  28 +-
 .../apache/solr/cloud/TestCloudDeleteByQuery.java  |  27 +-
 .../TestCloudPhrasesIdentificationComponent.java   |  27 +-
 .../solr/cloud/TestCloudPseudoReturnFields.java    |  33 +-
 .../solr/cloud/TestCloudSearcherWarming.java       |  28 +-
 .../cloud/TestDynamicFieldNamesIndexCorrectly.java |   3 +-
 .../cloud/TestLeaderElectionWithEmptyReplica.java  |   7 +-
 .../solr/cloud/TestOnReconnectListenerSupport.java |   1 -
 .../solr/cloud/TestPullReplicaErrorHandling.java   |   5 +-
 .../apache/solr/cloud/TestRandomFlRTGCloud.java    |  17 +-
 .../org/apache/solr/cloud/TestSegmentSorting.java  |  54 ++--
 .../cloud/TestStressCloudBlindAtomicUpdates.java   |  46 +--
 .../cloud/TestTolerantUpdateProcessorCloud.java    |  47 +--
 .../TestTolerantUpdateProcessorRandomCloud.java    |  31 +-
 .../solr/cloud/api/collections/ShardSplitTest.java |  14 +-
 .../cloud/api/collections/SplitByPrefixTest.java   |   3 +-
 .../component/TestTrackingShardHandlerFactory.java |  32 +-
 .../transform/TestSubQueryTransformerDistrib.java  |   1 -
 .../solr/search/CurrencyRangeFacetCloudTest.java   |  22 +-
 .../org/apache/solr/search/TestTaskManagement.java |  15 +-
 .../solr/search/facet/RangeFacetCloudTest.java     |  53 ++--
 .../search/facet/TestCloudJSONFacetJoinDomain.java |  12 +-
 .../solr/search/facet/TestCloudJSONFacetSKG.java   |  12 +-
 .../search/facet/TestCloudJSONFacetSKGEquiv.java   |  11 +-
 .../solr/search/join/TestCloudNestedDocsSort.java  |   6 +-
 .../solr/servlet/TestRequestRateLimiter.java       | 188 ++++++------
 .../solr/update/DeleteByIdWithRouterFieldTest.java |  12 +-
 .../apache/solr/util/TestSolrCLIRunExample.java    |  21 +-
 .../solr/hdfs/cloud/HdfsCollectionsApiTest.java    |  48 +--
 .../cloud/HdfsWriteToMultipleCollectionsTest.java  |   9 +-
 .../SharedFileSystemAutoReplicaFailoverTest.java   |  12 +-
 .../org/apache/solr/hdfs/cloud/StressHdfsTest.java |  21 +-
 .../org/apache/solr/ltr/TestLTROnSolrCloud.java    |   2 -
 .../deployment-guide/pages/enabling-ssl.adoc       |   3 +-
 .../examples/IndexingNestedDocuments.java          | 334 +++++++++++----------
 .../client/solrj/impl/CloudHttp2SolrClient.java    |   8 +
 .../client/solrj/impl/CloudLegacySolrClient.java   |   8 +
 .../solr/client/solrj/impl/CloudSolrClient.java    |  10 +-
 .../solrj/impl/CloudHttp2SolrClientTest.java       |  21 +-
 .../client/solrj/impl/CloudSolrClientTest.java     |  19 +-
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   | 110 +------
 .../cloud/AbstractBasicDistributedZk2TestBase.java |   2 -
 .../cloud/AbstractBasicDistributedZkTestBase.java  |  14 +-
 .../AbstractChaosMonkeyNothingIsSafeTestBase.java  |   7 +-
 .../solr/cloud/AbstractFullDistribZkTestBase.java  |  50 ++-
 .../solr/cloud/AbstractRecoveryZkTestBase.java     |  75 ++---
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |  56 +++-
 67 files changed, 1218 insertions(+), 1064 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 190cb4c9116..c849ee814c8 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -28,6 +28,9 @@ Improvements
   fetching single property values (`GET /api/aliases/aliasName/properties/propName`), and property deletion
   (`DELETE /api/aliases/aliasName/properties/propName`). (Alex Deparvu via Jason Gerlowski)
 
+* SOLR-10466: Introduce Builder setter for defaultCollection on cloud SolrClients.  Deprecated
+  direct setter setDefaultCollection on cloud SolrClients. (Eric Pugh, David Smiley)
+
 Optimizations
 ---------------------
 
@@ -40,7 +43,8 @@ Optimizations
 
 Bug Fixes
 ---------------------
-(No changes)
+
+* SOLR-16638: Fix Http2SolrClient's exception message when serverBaseUrl is null (Alex Deparvu via Kevin Risden)
 
 Dependency Upgrades
 ---------------------
diff --git a/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java b/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
index 837f6015983..ee990406165 100644
--- a/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
+++ b/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
@@ -19,6 +19,8 @@ package org.apache.solr.client.solrj.impl;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
+import java.util.Collections;
+import java.util.Optional;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -85,15 +87,21 @@ public class ConnectionReuseTest extends SolrCloudTestCase {
       case 1:
         return getHttpSolrClient(url.toString() + "/" + COLLECTION, httpClient);
       case 2:
-        CloudSolrClient client =
-            getCloudSolrClient(
-                cluster.getZkServer().getZkAddress(),
-                random().nextBoolean(),
-                httpClient,
-                30000,
-                60000);
-        client.setDefaultCollection(COLLECTION);
-        return client;
+        var builder =
+            new RandomizingCloudSolrClientBuilder(
+                Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty());
+        boolean shardLeadersOnly = random().nextBoolean();
+        if (shardLeadersOnly) {
+          builder.sendUpdatesOnlyToShardLeaders();
+        } else {
+          builder.sendUpdatesToAllReplicasInShard();
+        }
+        builder.withDefaultCollection(COLLECTION);
+        return builder
+            .withHttpClient(httpClient)
+            .withConnectionTimeout(30000)
+            .withSocketTimeout(60000)
+            .build();
     }
     throw new RuntimeException("impossible");
   }
diff --git a/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
index 0e7be0e6544..a4061f7ad87 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
@@ -947,9 +947,14 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
       responseConsumer.accept(cluster.getSolrClient().query(collectionList, solrQuery));
     } else {
       // new CloudSolrClient (random shardLeadersOnly)
-      try (CloudSolrClient solrClient = getCloudSolrClient(cluster)) {
-        if (random().nextBoolean()) {
-          solrClient.setDefaultCollection(collectionList);
+
+      RandomizingCloudSolrClientBuilder builder = new RandomizingCloudSolrClientBuilder(cluster);
+      boolean useDefaultCollection = random().nextBoolean();
+      try (CloudSolrClient solrClient =
+          useDefaultCollection
+              ? builder.withDefaultCollection(collectionList).build()
+              : builder.build()) {
+        if (useDefaultCollection) {
           responseConsumer.accept(solrClient.query(null, solrQuery));
         } else {
           responseConsumer.accept(solrClient.query(collectionList, solrQuery));
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
index 0ecb4aab598..ddb72aad118 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPullReplicasTest.java
@@ -145,10 +145,8 @@ public class ChaosMonkeyNothingIsSafeWithPullReplicasTest extends AbstractFullDi
   }
 
   protected CloudSolrClient createCloudClient(String defaultCollection, int socketTimeout) {
-    CloudSolrClient client =
-        getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean(), 30000, socketTimeout);
-    if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
-    return client;
+    return getCloudSolrClient(
+        zkServer.getZkAddress(), defaultCollection, random().nextBoolean(), 30000, socketTimeout);
   }
 
   @Test
diff --git a/solr/core/src/test/org/apache/solr/cloud/CleanupOldIndexTest.java b/solr/core/src/test/org/apache/solr/cloud/CleanupOldIndexTest.java
index 7fd80c33670..deb828c8aa4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CleanupOldIndexTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CleanupOldIndexTest.java
@@ -59,13 +59,12 @@ public class CleanupOldIndexTest extends SolrCloudTestCase {
 
     CollectionAdminRequest.createCollection(COLLECTION, "conf1", 1, 2)
         .processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
-    // TODO make this configurable on StoppableIndexingThread
-    cluster.getSolrClient().setDefaultCollection(COLLECTION);
 
     int maxDoc = atLeast(300);
 
     StoppableIndexingThread indexThread =
-        new StoppableIndexingThread(null, cluster.getSolrClient(), "1", true, maxDoc, 1, true);
+        new StoppableIndexingThread(
+            null, cluster.getSolrClient(COLLECTION), "1", true, maxDoc, 1, true);
     indexThread.start();
 
     // give some time to index...
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index 40b822a3773..30d9339d50c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -823,11 +823,10 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
   @Test
   public void testReadOnlyCollection() throws Exception {
     String collectionName = getSaferTestName();
-    CloudSolrClient solrClient = cluster.getSolrClient();
-
-    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2).process(solrClient);
 
-    solrClient.setDefaultCollection(collectionName);
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
+        .process(cluster.getSolrClient());
+    CloudSolrClient solrClient = cluster.getSolrClient(collectionName);
 
     cluster.waitForActiveCollection(collectionName, 2, 4);
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java b/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
index f7947c5f2e4..5937ed75cfd 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
@@ -153,8 +153,6 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
     SchemaResponse.UpdateResponse multipleUpdatesResponse =
         multiUpdateRequest.process(cluster.getSolrClient(), COLLECTION);
     assertNull("Error adding fields", multipleUpdatesResponse.getResponse().get("errors"));
-
-    cluster.getSolrClient().setDefaultCollection(COLLECTION);
   }
 
   @AfterClass
@@ -164,7 +162,7 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
 
   @Before
   public void clean() throws IOException, SolrServerException {
-    CloudSolrClient client = cluster.getSolrClient();
+    CloudSolrClient client = cluster.getSolrClient(COLLECTION);
     client.deleteByQuery("*:*");
     client.commit();
     resetFields(fieldsToTestSingle);
@@ -184,7 +182,7 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
     // For this test, I want to ensure that there are shards that do _not_ have a doc with any of
     // the DV_only fields, see SOLR-5260. So I'll add exactly 1 document to a 4 shard collection.
 
-    CloudSolrClient client = cluster.getSolrClient();
+    CloudSolrClient client = cluster.getSolrClient(COLLECTION);
 
     SolrInputDocument doc = new SolrInputDocument();
     doc.addField("id", "1");
@@ -225,7 +223,7 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
   // one server.
   @Test
   public void testGroupingSorting() throws IOException, SolrServerException {
-    CloudSolrClient client = cluster.getSolrClient();
+    CloudSolrClient client = cluster.getSolrClient(COLLECTION);
 
     // The point of these is to have at least one shard w/o the value.
     // While getting values for each of these fields starts _out_ random, each successive
diff --git a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
index bec70654c65..c3008437440 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
@@ -21,9 +21,11 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.cloud.SocketProxy;
+import org.apache.solr.client.solrj.impl.CloudLegacySolrClient;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.SolrException;
@@ -44,6 +46,7 @@ import org.slf4j.LoggerFactory;
 
 @Nightly // this test is currently too slow for non-nightly
 public class ForceLeaderTest extends HttpPartitionTest {
+  public static final String TEST_COLLECTION = "forceleader_lower_terms_collection";
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   @BeforeClass
@@ -69,39 +72,37 @@ public class ForceLeaderTest extends HttpPartitionTest {
     handle.put("maxScore", SKIPVAL);
     handle.put("timestamp", SKIPVAL);
 
-    String testCollectionName = "forceleader_lower_terms_collection";
-    createCollection(testCollectionName, "conf1", 1, 3);
+    createCollection(TEST_COLLECTION, "conf1", 1, 3);
 
     try {
-      cloudClient.setDefaultCollection(testCollectionName);
       List<Replica> notLeaders =
-          ensureAllReplicasAreActive(testCollectionName, SHARD1, 1, 3, maxWaitSecsToSeeAllActive);
+          ensureAllReplicasAreActive(TEST_COLLECTION, SHARD1, 1, 3, maxWaitSecsToSeeAllActive);
       assertEquals(
           "Expected 2 replicas for collection "
-              + testCollectionName
+              + TEST_COLLECTION
               + " but found "
               + notLeaders.size()
               + "; clusterState: "
-              + printClusterStateInfo(testCollectionName),
+              + printClusterStateInfo(TEST_COLLECTION),
           2,
           notLeaders.size());
 
-      Replica leader = ZkStateReader.from(cloudClient).getLeaderRetry(testCollectionName, SHARD1);
+      Replica leader = ZkStateReader.from(cloudClient).getLeaderRetry(TEST_COLLECTION, SHARD1);
       JettySolrRunner notLeader0 = getJettyOnPort(getReplicaPort(notLeaders.get(0)));
       ZkController zkController = notLeader0.getCoreContainer().getZkController();
 
       if (log.isInfoEnabled()) {
         log.info("Before put non leaders into lower term: {}", printClusterStateInfo());
       }
-      putNonLeadersIntoLowerTerm(testCollectionName, SHARD1, zkController, leader, notLeaders);
+      putNonLeadersIntoLowerTerm(TEST_COLLECTION, SHARD1, zkController, leader, notLeaders);
 
       for (Replica replica : notLeaders) {
-        waitForState(testCollectionName, replica.getName(), State.DOWN, 60000);
+        waitForState(TEST_COLLECTION, replica.getName(), State.DOWN, 60000);
       }
-      waitForState(testCollectionName, leader.getName(), State.DOWN, 60000);
-      ZkStateReader.from(cloudClient).forceUpdateCollection(testCollectionName);
+      waitForState(TEST_COLLECTION, leader.getName(), State.DOWN, 60000);
+      ZkStateReader.from(cloudClient).forceUpdateCollection(TEST_COLLECTION);
       ClusterState clusterState = cloudClient.getClusterState();
-      int numActiveReplicas = getNumberOfActiveReplicas(clusterState, testCollectionName, SHARD1);
+      int numActiveReplicas = getNumberOfActiveReplicas(clusterState, TEST_COLLECTION, SHARD1);
       assertEquals(
           "Expected only 0 active replica but found "
               + numActiveReplicas
@@ -112,7 +113,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
 
       int numReplicasOnLiveNodes = 0;
       for (Replica rep :
-          clusterState.getCollection(testCollectionName).getSlice(SHARD1).getReplicas()) {
+          clusterState.getCollection(TEST_COLLECTION).getSlice(SHARD1).getReplicas()) {
         if (clusterState.getLiveNodes().contains(rep.getNodeName())) {
           numReplicasOnLiveNodes++;
         }
@@ -124,43 +125,43 @@ public class ForceLeaderTest extends HttpPartitionTest {
       // Assert there is no leader yet
       assertNull(
           "Expected no leader right now. State: "
-              + clusterState.getCollection(testCollectionName).getSlice(SHARD1),
-          clusterState.getCollection(testCollectionName).getSlice(SHARD1).getLeader());
+              + clusterState.getCollection(TEST_COLLECTION).getSlice(SHARD1),
+          clusterState.getCollection(TEST_COLLECTION).getSlice(SHARD1).getLeader());
 
-      assertSendDocFails(3);
+      assertSendDocFails(TEST_COLLECTION, 3);
 
       log.info("Do force leader...");
-      doForceLeader(testCollectionName, SHARD1);
+      doForceLeader(TEST_COLLECTION, SHARD1);
 
       // By now we have an active leader. Wait for recoveries to begin
-      waitForRecoveriesToFinish(testCollectionName, ZkStateReader.from(cloudClient), true);
+      waitForRecoveriesToFinish(TEST_COLLECTION, ZkStateReader.from(cloudClient), true);
 
-      ZkStateReader.from(cloudClient).forceUpdateCollection(testCollectionName);
+      ZkStateReader.from(cloudClient).forceUpdateCollection(TEST_COLLECTION);
       clusterState = cloudClient.getClusterState();
       if (log.isInfoEnabled()) {
         log.info(
             "After forcing leader: {}",
-            clusterState.getCollection(testCollectionName).getSlice(SHARD1));
+            clusterState.getCollection(TEST_COLLECTION).getSlice(SHARD1));
       }
       // we have a leader
       Replica newLeader =
-          clusterState.getCollectionOrNull(testCollectionName).getSlice(SHARD1).getLeader();
+          clusterState.getCollectionOrNull(TEST_COLLECTION).getSlice(SHARD1).getLeader();
       assertNotNull(newLeader);
       // leader is active
       assertEquals(State.ACTIVE, newLeader.getState());
 
-      numActiveReplicas = getNumberOfActiveReplicas(clusterState, testCollectionName, SHARD1);
+      numActiveReplicas = getNumberOfActiveReplicas(clusterState, TEST_COLLECTION, SHARD1);
       assertEquals(2, numActiveReplicas);
 
       // Assert that indexing works again
       log.info("Sending doc 4...");
-      sendDoc(4);
+      sendDoc(TEST_COLLECTION, 4);
       log.info("Committing...");
-      cloudClient.commit();
+      cloudClient.commit(TEST_COLLECTION);
       log.info("Doc 4 sent and commit issued");
 
-      assertDocsExistInAllReplicas(notLeaders, testCollectionName, 1, 1);
-      assertDocsExistInAllReplicas(notLeaders, testCollectionName, 4, 4);
+      assertDocsExistInAllReplicas(notLeaders, TEST_COLLECTION, 1, 1);
+      assertDocsExistInAllReplicas(notLeaders, TEST_COLLECTION, 4, 4);
 
       if (useTlogReplicas()) {}
       // Docs 1 and 4 should be here. 2 was lost during the partition, 3 had failed to be indexed.
@@ -173,7 +174,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
             "Expected only 2 documents in the index",
             () -> {
               try {
-                return 2 == cloudClient.query(params).getResults().getNumFound();
+                return 2 == cloudClient.query(TEST_COLLECTION, params).getResults().getNumFound();
               } catch (Exception e) {
                 return false;
               }
@@ -182,17 +183,33 @@ public class ForceLeaderTest extends HttpPartitionTest {
         assertEquals(
             "Expected only 2 documents in the index",
             2,
-            cloudClient.query(params).getResults().getNumFound());
+            cloudClient.query(TEST_COLLECTION, params).getResults().getNumFound());
       }
 
-      bringBackOldLeaderAndSendDoc(testCollectionName, leader, notLeaders, 5);
+      bringBackOldLeaderAndSendDoc(TEST_COLLECTION, leader, notLeaders, 5);
     } finally {
       log.info("Cleaning up after the test.");
       // try to clean up
-      attemptCollectionDelete(cloudClient, testCollectionName);
+      attemptCollectionDelete(cloudClient, TEST_COLLECTION);
     }
   }
 
+  /**
+   * For this test, we need a cloudClient that is not randomized since we need to NEVER send the
+   * updates only to the leader. The way the RandomizingCloudSolrClientBuilder works, you can't
+   * avoid its internal decision-making process to sometimes send updates only to leaders. We
+   * override the definition of this class in AbstractFullDistribZkTestBase to make sure we always
+   * use DEFAULT_COLLECTION defined in ForceLeaderTest.
+   */
+  @Override
+  protected CloudSolrClient createCloudClient(String defaultCollection) {
+    CloudLegacySolrClient.Builder builder =
+        new CloudLegacySolrClient.Builder(
+            Collections.singletonList(zkServer.getZkAddress()), Optional.empty());
+    builder.withDefaultCollection(TEST_COLLECTION);
+    return builder.withConnectionTimeout(30000).withSocketTimeout(120000).build();
+  }
+
   private void putNonLeadersIntoLowerTerm(
       String collectionName,
       String shard,
@@ -204,7 +221,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
     for (int i = 0; i < notLeaders.size(); i++)
       nonLeaderProxies[i] = getProxyForReplica(notLeaders.get(i));
 
-    sendDoc(1);
+    sendDoc(collectionName, 1);
 
     // ok, now introduce a network partition between the leader and both replicas
     log.info("Closing proxies for the non-leader replicas...");
@@ -214,7 +231,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
     // indexing during a partition
     log.info("Sending a doc during the network partition...");
     JettySolrRunner leaderJetty = getJettyOnPort(getReplicaPort(leader));
-    sendDoc(2, leaderJetty);
+    sendDoc(collectionName, 2, leaderJetty);
 
     for (Replica replica : notLeaders) {
       waitForState(collectionName, replica.getName(), State.DOWN, 60000);
@@ -258,12 +275,12 @@ public class ForceLeaderTest extends HttpPartitionTest {
     }
   }
 
-  private void assertSendDocFails(int docId) {
+  private void assertSendDocFails(String collectionName, int docId) {
     // sending a doc in this state fails
     expectThrows(
         SolrException.class,
         "Should've failed indexing during a down state.",
-        () -> sendDoc(docId));
+        () -> sendDoc(collectionName, docId));
   }
 
   private void bringBackOldLeaderAndSendDoc(
@@ -284,29 +301,34 @@ public class ForceLeaderTest extends HttpPartitionTest {
     int numActiveReplicas = getNumberOfActiveReplicas(clusterState, collection, SHARD1);
     assertEquals(1 + notLeaders.size(), numActiveReplicas);
     log.info("Sending doc {}...", docid);
-    sendDoc(docid);
+    sendDoc(collection, docid);
     log.info("Committing...");
-    cloudClient.commit();
+    cloudClient.commit(collection);
     log.info("Doc {} sent and commit issued", docid);
     assertDocsExistInAllReplicas(notLeaders, collection, docid, docid);
     assertDocsExistInAllReplicas(Collections.singletonList(leader), collection, docid, docid);
   }
 
   @Override
-  protected int sendDoc(int docId) throws Exception {
+  protected int sendDoc(String collectionName, int docId) throws Exception {
     SolrInputDocument doc = new SolrInputDocument();
     doc.addField(id, String.valueOf(docId));
     doc.addField("a_t", "hello" + docId);
 
-    return sendDocsWithRetry(Collections.singletonList(doc), 1, 5, 1);
+    return sendDocsWithRetry(collectionName, Collections.singletonList(doc), 1, 5, 1);
   }
 
   private void doForceLeader(String collectionName, String shard)
       throws IOException, SolrServerException {
     CollectionAdminRequest.ForceLeader forceLeader =
         CollectionAdminRequest.forceLeaderElection(collectionName, shard);
+
     try (CloudSolrClient cloudClient =
-        getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean(), 30000, 60000)) {
+        new CloudLegacySolrClient.Builder(
+                Collections.singletonList(zkServer.getZkAddress()), Optional.empty())
+            .withConnectionTimeout(3000)
+            .withSocketTimeout(60000)
+            .build()) {
       cloudClient.request(forceLeader);
     }
   }
diff --git a/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java b/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
index ec00cabd6c7..5ad626d7a79 100644
--- a/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
@@ -40,12 +40,12 @@ import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.cloud.CollectionStatePredicate;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Replica.ReplicaStateProps;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.embedded.JettySolrRunner;
@@ -70,16 +70,14 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
   @After
   public void purgeAllCollections() throws Exception {
     cluster.deleteAllCollections();
-    cluster.getSolrClient().setDefaultCollection(null);
   }
 
   /**
-   * Creates a new 2x2 collection using a unique name, blocking until it's state is fully active,
-   * and sets that collection as the default on the cluster's default CloudSolrClient.
+   * Creates a new 2x2 collection using a unique name, blocking until it's state is fully active.
    *
    * @return the name of the new collection
    */
-  public static String createAndSetNewDefaultCollection() throws Exception {
+  public static String createNewCollection() throws Exception {
     final CloudSolrClient cloudClient = cluster.getSolrClient();
     final String name = "test_collection_" + NAME_COUNTER.getAndIncrement();
     assertTrue(
@@ -92,73 +90,86 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
             DEFAULT_TIMEOUT,
             TimeUnit.SECONDS,
             (n, c) -> DocCollection.isFullyActive(n, c, 2, 2));
-    cloudClient.setDefaultCollection(name);
     return name;
   }
 
   @Test
   public void testBasicUpdates() throws Exception {
+    final String collectionName = createNewCollection();
     final CloudSolrClient cloudClient = cluster.getSolrClient();
-    final String collectionName = createAndSetNewDefaultCollection();
 
     // add a doc, update it, and delete it
-    addUpdateDelete("doc1");
-    assertEquals(0, cloudClient.query(params("q", "*:*")).getResults().getNumFound());
+    addUpdateDelete(collectionName, "doc1");
+    assertEquals(
+        0, cloudClient.query(collectionName, params("q", "*:*")).getResults().getNumFound());
 
     // add 2 docs in a single request
-    addTwoDocsInOneRequest("doc2", "doc3");
-    assertEquals(2, cloudClient.query(params("q", "*:*")).getResults().getNumFound());
+    addTwoDocsInOneRequest(collectionName, "doc2", "doc3");
+    assertEquals(
+        2, cloudClient.query(collectionName, params("q", "*:*")).getResults().getNumFound());
 
     // 2 deletes in a single request...
     assertEquals(
         0,
         (new UpdateRequest().deleteById("doc2").deleteById("doc3"))
-            .process(cloudClient)
+            .process(cloudClient, collectionName)
             .getStatus());
-    assertEquals(0, cloudClient.commit().getStatus());
+    assertEquals(0, cloudClient.commit(collectionName).getStatus());
 
-    assertEquals(0, cloudClient.query(params("q", "*:*")).getResults().getNumFound());
+    assertEquals(
+        0, cloudClient.query(collectionName, params("q", "*:*")).getResults().getNumFound());
 
     // add a doc that we will then delete later after adding two other docs (all before next
     // commit).
     assertEquals(
-        0, cloudClient.add(sdoc("id", "doc4", "content_s", "will_delete_later")).getStatus());
-    assertEquals(0, cloudClient.add(sdocs(sdoc("id", "doc5"), sdoc("id", "doc6"))).getStatus());
-    assertEquals(0, cloudClient.deleteById("doc4").getStatus());
-    assertEquals(0, cloudClient.commit().getStatus());
+        0,
+        cloudClient
+            .add(collectionName, sdoc("id", "doc4", "content_s", "will_delete_later"))
+            .getStatus());
+    assertEquals(
+        0,
+        cloudClient.add(collectionName, sdocs(sdoc("id", "doc5"), sdoc("id", "doc6"))).getStatus());
+    assertEquals(0, cloudClient.deleteById(collectionName, "doc4").getStatus());
+    assertEquals(0, cloudClient.commit(collectionName).getStatus());
 
-    assertEquals(0, cloudClient.query(params("q", "id:doc4")).getResults().getNumFound());
-    assertEquals(1, cloudClient.query(params("q", "id:doc5")).getResults().getNumFound());
-    assertEquals(1, cloudClient.query(params("q", "id:doc6")).getResults().getNumFound());
-    assertEquals(2, cloudClient.query(params("q", "*:*")).getResults().getNumFound());
+    assertEquals(
+        0, cloudClient.query(collectionName, params("q", "id:doc4")).getResults().getNumFound());
+    assertEquals(
+        1, cloudClient.query(collectionName, params("q", "id:doc5")).getResults().getNumFound());
+    assertEquals(
+        1, cloudClient.query(collectionName, params("q", "id:doc6")).getResults().getNumFound());
+    assertEquals(
+        2, cloudClient.query(collectionName, params("q", "*:*")).getResults().getNumFound());
 
-    checkShardConsistency(params("q", "*:*", "rows", "9999", "_trace", "post_doc_5_6"));
+    checkShardConsistency(
+        collectionName, params("q", "*:*", "rows", "9999", "_trace", "post_doc_5_6"));
 
     // delete everything....
-    assertEquals(0, cloudClient.deleteByQuery("*:*").getStatus());
-    assertEquals(0, cloudClient.commit().getStatus());
-    assertEquals(0, cloudClient.query(params("q", "*:*")).getResults().getNumFound());
+    assertEquals(0, cloudClient.deleteByQuery(collectionName, "*:*").getStatus());
+    assertEquals(0, cloudClient.commit(collectionName).getStatus());
+    assertEquals(
+        0, cloudClient.query(collectionName, params("q", "*:*")).getResults().getNumFound());
 
-    checkShardConsistency(params("q", "*:*", "rows", "9999", "_trace", "delAll"));
+    checkShardConsistency(collectionName, params("q", "*:*", "rows", "9999", "_trace", "delAll"));
   }
 
   public void testDeleteByIdImplicitRouter() throws Exception {
     final CloudSolrClient cloudClient = cluster.getSolrClient();
-    final String name = "implicit_collection_without_routerfield_" + NAME_COUNTER.getAndIncrement();
+    final String testCollectionName =
+        "implicit_collection_without_routerfield_" + NAME_COUNTER.getAndIncrement();
     assertEquals(
         RequestStatusState.COMPLETED,
         CollectionAdminRequest.createCollectionWithImplicitRouter(
-                name, "_default", "shard1,shard2", 2)
+                testCollectionName, "_default", "shard1,shard2", 2)
             .processAndWait(cloudClient, DEFAULT_TIMEOUT));
     ZkStateReader.from(cloudClient)
         .waitForState(
-            name,
-            (long) DEFAULT_TIMEOUT,
+            testCollectionName,
+            DEFAULT_TIMEOUT,
             TimeUnit.SECONDS,
-            (CollectionStatePredicate) (n, c1) -> DocCollection.isFullyActive(n, c1, 2, 2));
-    cloudClient.setDefaultCollection(name);
+            (n, c1) -> DocCollection.isFullyActive(n, c1, 2, 2));
 
-    final DocCollection docCol = cloudClient.getClusterState().getCollection(name);
+    final DocCollection docCol = cloudClient.getClusterState().getCollection(testCollectionName);
     try (SolrClient shard1 = getHttpSolrClient(docCol.getSlice("shard1").getLeader().getCoreUrl());
         SolrClient shard2 = getHttpSolrClient(docCol.getSlice("shard2").getLeader().getCoreUrl())) {
 
@@ -183,22 +194,25 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
               // including cloudClient helps us test view from other nodes that aren't the
               // leaders...
               for (SolrClient c : Arrays.asList(cloudClient, shard1, shard2)) {
+
+                ModifiableSolrParams params = params("q", "*:*");
+                if (c instanceof CloudSolrClient) {
+                  params.add("collection", testCollectionName);
+                }
                 assertEquals(
                     docCounts1.get() + docCounts2.get(),
-                    c.query(params("q", "*:*")).getResults().getNumFound());
+                    c.query(params).getResults().getNumFound());
 
                 assertEquals(
                     docCounts1.get(),
-                    c.query(params("q", "*:*", "shards", "shard1")).getResults().getNumFound());
+                    c.query(params.set("shards", "shard1")).getResults().getNumFound());
                 assertEquals(
                     docCounts2.get(),
-                    c.query(params("q", "*:*", "shards", "shard2")).getResults().getNumFound());
+                    c.query(params.set("shards", "shard2")).getResults().getNumFound());
 
                 assertEquals(
                     docCounts1.get() + docCounts2.get(),
-                    c.query(params("q", "*:*", "shards", "shard2,shard1"))
-                        .getResults()
-                        .getNumFound());
+                    c.query(params.set("shards", "shard2,shard1")).getResults().getNumFound());
               }
 
               assertEquals(
@@ -260,23 +274,23 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
 
   public void testDeleteByIdCompositeRouterWithRouterField() throws Exception {
     final CloudSolrClient cloudClient = cluster.getSolrClient();
-    final String name = "composite_collection_with_routerfield_" + NAME_COUNTER.getAndIncrement();
+    final String testCollectionName =
+        "composite_collection_with_routerfield_" + NAME_COUNTER.getAndIncrement();
     assertEquals(
         RequestStatusState.COMPLETED,
-        CollectionAdminRequest.createCollection(name, "_default", 2, 2)
+        CollectionAdminRequest.createCollection(testCollectionName, "_default", 2, 2)
             .setRouterName("compositeId")
             .setRouterField("routefield_s")
             .setShards("shard1,shard2")
             .processAndWait(cloudClient, DEFAULT_TIMEOUT));
     ZkStateReader.from(cloudClient)
         .waitForState(
-            name,
+            testCollectionName,
             DEFAULT_TIMEOUT,
             TimeUnit.SECONDS,
             (n, c1) -> DocCollection.isFullyActive(n, c1, 2, 2));
-    cloudClient.setDefaultCollection(name);
 
-    final DocCollection docCol = cloudClient.getClusterState().getCollection(name);
+    final DocCollection docCol = cloudClient.getClusterState().getCollection(testCollectionName);
     try (SolrClient shard1 = getHttpSolrClient(docCol.getSlice("shard1").getLeader().getCoreUrl());
         SolrClient shard2 = getHttpSolrClient(docCol.getSlice("shard2").getLeader().getCoreUrl())) {
 
@@ -307,16 +321,21 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
               // including cloudClient helps us test view from other nodes that aren't the
               // leaders...
               for (SolrClient c : Arrays.asList(cloudClient, shard1, shard2)) {
+
+                ModifiableSolrParams params = params("q", "*:*");
+                if (c instanceof CloudSolrClient) {
+                  params.add("collection", testCollectionName);
+                }
                 assertEquals(
                     docCountsEurope.get() + docCountsAfrica.get(),
-                    c.query(params("q", "*:*")).getResults().getNumFound());
+                    c.query(params).getResults().getNumFound());
 
                 assertEquals(
                     docCountsEurope.get(),
-                    c.query(params("q", "*:*", "_route_", "europe")).getResults().getNumFound());
+                    c.query(params.set("_route_", "europe")).getResults().getNumFound());
                 assertEquals(
                     docCountsAfrica.get(),
-                    c.query(params("q", "*:*", "_route_", "africa")).getResults().getNumFound());
+                    c.query(params.set("_route_", "africa")).getResults().getNumFound());
               }
             } catch (Exception sse) {
               throw new RuntimeException(sse);
@@ -373,7 +392,6 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
   public void testThatCantForwardToLeaderFails() throws Exception {
     final CloudSolrClient cloudClient = cluster.getSolrClient();
     final String collectionName = "test_collection_" + NAME_COUNTER.getAndIncrement();
-    cloudClient.setDefaultCollection(collectionName);
 
     // get a random node for use in our collection before creating the one we'll partition
     final JettySolrRunner otherLeader = cluster.getRandomJetty(random());
@@ -480,72 +498,106 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
   }
 
   /** NOTE: uses the cluster's CloudSolrClient and assumes default collection has been set */
-  private void addTwoDocsInOneRequest(String docIdA, String docIdB) throws Exception {
+  private void addTwoDocsInOneRequest(String collectionName, String docIdA, String docIdB)
+      throws Exception {
     final CloudSolrClient cloudClient = cluster.getSolrClient();
+    assertNotNull(collectionName);
 
-    assertEquals(0, cloudClient.add(sdocs(sdoc("id", docIdA), sdoc("id", docIdB))).getStatus());
-    assertEquals(0, cloudClient.commit().getStatus());
+    assertEquals(
+        0,
+        cloudClient.add(collectionName, sdocs(sdoc("id", docIdA), sdoc("id", docIdB))).getStatus());
+    assertEquals(0, cloudClient.commit(collectionName).getStatus());
 
     assertEquals(
         2,
         cloudClient
-            .query(params("q", "id:(" + docIdA + " OR " + docIdB + ")"))
+            .query(collectionName, params("q", "id:(" + docIdA + " OR " + docIdB + ")"))
             .getResults()
             .getNumFound());
 
-    checkShardConsistency(params("q", "*:*", "rows", "99", "_trace", "two_docs"));
+    checkShardConsistency(collectionName, params("q", "*:*", "rows", "99", "_trace", "two_docs"));
   }
 
   /** NOTE: uses the cluster's CloudSolrClient and assumes default collection has been set */
-  private void addUpdateDelete(String docId) throws Exception {
+  private void addUpdateDelete(String collectionName, String docId) throws Exception {
     final CloudSolrClient cloudClient = cluster.getSolrClient();
 
     // add the doc, confirm we can query it...
-    assertEquals(0, cloudClient.add(sdoc("id", docId, "content_t", "originalcontent")).getStatus());
-    assertEquals(0, cloudClient.commit().getStatus());
+    assertEquals(
+        0,
+        cloudClient
+            .add(collectionName, sdoc("id", docId, "content_t", "originalcontent"))
+            .getStatus());
+    assertEquals(0, cloudClient.commit(collectionName).getStatus());
 
-    assertEquals(1, cloudClient.query(params("q", "id:" + docId)).getResults().getNumFound());
     assertEquals(
-        1, cloudClient.query(params("q", "content_t:originalcontent")).getResults().getNumFound());
+        1,
+        cloudClient.query(collectionName, params("q", "id:" + docId)).getResults().getNumFound());
+    assertEquals(
+        1,
+        cloudClient
+            .query(collectionName, params("q", "content_t:originalcontent"))
+            .getResults()
+            .getNumFound());
     assertEquals(
         1,
         cloudClient
-            .query(params("q", "content_t:originalcontent AND id:" + docId))
+            .query(collectionName, params("q", "content_t:originalcontent AND id:" + docId))
             .getResults()
             .getNumFound());
 
-    checkShardConsistency(params("q", "id:" + docId, "rows", "99", "_trace", "original_doc"));
+    checkShardConsistency(
+        collectionName, params("q", "id:" + docId, "rows", "99", "_trace", "original_doc"));
 
     // update doc
-    assertEquals(0, cloudClient.add(sdoc("id", docId, "content_t", "updatedcontent")).getStatus());
-    assertEquals(0, cloudClient.commit().getStatus());
+    assertEquals(
+        0,
+        cloudClient
+            .add(collectionName, sdoc("id", docId, "content_t", "updatedcontent"))
+            .getStatus());
+    assertEquals(0, cloudClient.commit(collectionName).getStatus());
 
     // confirm we can query the doc by updated content and not original
     assertEquals(
-        0, cloudClient.query(params("q", "content_t:originalcontent")).getResults().getNumFound());
+        0,
+        cloudClient
+            .query(collectionName, params("q", "content_t:originalcontent"))
+            .getResults()
+            .getNumFound());
     assertEquals(
-        1, cloudClient.query(params("q", "content_t:updatedcontent")).getResults().getNumFound());
+        1,
+        cloudClient
+            .query(collectionName, params("q", "content_t:updatedcontent"))
+            .getResults()
+            .getNumFound());
     assertEquals(
         1,
         cloudClient
-            .query(params("q", "content_t:updatedcontent AND id:" + docId))
+            .query(collectionName, params("q", "content_t:updatedcontent AND id:" + docId))
             .getResults()
             .getNumFound());
 
     // delete the doc, confirm it no longer matches in queries
-    assertEquals(0, cloudClient.deleteById(docId).getStatus());
-    assertEquals(0, cloudClient.commit().getStatus());
+    assertEquals(0, cloudClient.deleteById(collectionName, docId).getStatus());
+    assertEquals(0, cloudClient.commit(collectionName).getStatus());
 
-    assertEquals(0, cloudClient.query(params("q", "id:" + docId)).getResults().getNumFound());
     assertEquals(
-        0, cloudClient.query(params("q", "content_t:updatedcontent")).getResults().getNumFound());
+        0,
+        cloudClient.query(collectionName, params("q", "id:" + docId)).getResults().getNumFound());
+    assertEquals(
+        0,
+        cloudClient
+            .query(collectionName, params("q", "content_t:updatedcontent"))
+            .getResults()
+            .getNumFound());
 
-    checkShardConsistency(params("q", "id:" + docId, "rows", "99", "_trace", "del_updated_doc"));
+    checkShardConsistency(
+        collectionName, params("q", "id:" + docId, "rows", "99", "_trace", "del_updated_doc"));
   }
 
   public long testIndexQueryDeleteHierarchical() throws Exception {
     final CloudSolrClient cloudClient = cluster.getSolrClient();
-    final String collectionName = createAndSetNewDefaultCollection();
+    final String collectionName = createNewCollection();
 
     // index
     long docId = 42;
@@ -564,23 +616,31 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
       }
 
       uReq.add(topDocument);
-      assertEquals(i + "/" + docId, 0, uReq.process(cloudClient).getStatus());
+      assertEquals(i + "/" + docId, 0, uReq.process(cloudClient, collectionName).getStatus());
     }
-    assertEquals(0, cloudClient.commit().getStatus());
+    assertEquals(0, cloudClient.commit(collectionName).getStatus());
 
     checkShardConsistency(
+        collectionName,
         params("q", "*:*", "rows", "9999", "_trace", "added_all_top_docs_with_kids"));
 
     // query
 
     // parents
     assertEquals(
-        topDocsNum, cloudClient.query(new SolrQuery("type_s:parent")).getResults().getNumFound());
+        topDocsNum,
+        cloudClient
+            .query(collectionName, new SolrQuery("type_s:parent"))
+            .getResults()
+            .getNumFound());
 
     // children
     assertEquals(
         topDocsNum * childsNum,
-        cloudClient.query(new SolrQuery("type_s:child")).getResults().getNumFound());
+        cloudClient
+            .query(collectionName, new SolrQuery("type_s:child"))
+            .getResults()
+            .getNumFound());
 
     // grandchildren
     //
@@ -589,14 +649,18 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
     // x = 2 * ((t-1) * t / 2) = t * (t - 1)
     assertEquals(
         topDocsNum * childsNum * (childsNum - 1),
-        cloudClient.query(new SolrQuery("type_s:grand")).getResults().getNumFound());
+        cloudClient
+            .query(collectionName, new SolrQuery("type_s:grand"))
+            .getResults()
+            .getNumFound());
 
     // delete
-    assertEquals(0, cloudClient.deleteByQuery("*:*").getStatus());
-    assertEquals(0, cloudClient.commit().getStatus());
-    assertEquals(0, cloudClient.query(params("q", "*:*")).getResults().getNumFound());
+    assertEquals(0, cloudClient.deleteByQuery(collectionName, "*:*").getStatus());
+    assertEquals(0, cloudClient.commit(collectionName).getStatus());
+    assertEquals(
+        0, cloudClient.query(collectionName, params("q", "*:*")).getResults().getNumFound());
 
-    checkShardConsistency(params("q", "*:*", "rows", "9999", "_trace", "delAll"));
+    checkShardConsistency(collectionName, params("q", "*:*", "rows", "9999", "_trace", "delAll"));
 
     return docId;
   }
@@ -622,27 +686,30 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
 
   public void testIndexingOneDocPerRequestWithHttpSolrClient() throws Exception {
     final CloudSolrClient cloudClient = cluster.getSolrClient();
-    final String collectionName = createAndSetNewDefaultCollection();
+    final String collectionName = createNewCollection();
 
     final int numDocs = atLeast(50);
     for (int i = 0; i < numDocs; i++) {
-      UpdateRequest uReq;
-      uReq = new UpdateRequest();
       assertEquals(
           0,
           cloudClient
-              .add(sdoc("id", i, "text_t", TestUtil.randomRealisticUnicodeString(random(), 200)))
+              .add(
+                  collectionName,
+                  sdoc("id", i, "text_t", TestUtil.randomRealisticUnicodeString(random(), 200)))
               .getStatus());
     }
-    assertEquals(0, cloudClient.commit().getStatus());
-    assertEquals(numDocs, cloudClient.query(params("q", "*:*")).getResults().getNumFound());
+    assertEquals(0, cloudClient.commit(collectionName).getStatus());
+    assertEquals(
+        numDocs, cloudClient.query(collectionName, params("q", "*:*")).getResults().getNumFound());
 
-    checkShardConsistency(params("q", "*:*", "rows", "" + (1 + numDocs), "_trace", "addAll"));
+    checkShardConsistency(
+        collectionName, params("q", "*:*", "rows", "" + (1 + numDocs), "_trace", "addAll"));
   }
 
   public void testIndexingBatchPerRequestWithHttpSolrClient() throws Exception {
-    final CloudSolrClient cloudClient = cluster.getSolrClient();
-    final String collectionName = createAndSetNewDefaultCollection();
+
+    final String collectionName = createNewCollection();
+    final CloudSolrClient cloudClient = cluster.getSolrClient(collectionName);
 
     final int numDocsPerBatch = atLeast(5);
     final int numBatchesPerThread = atLeast(5);
@@ -680,7 +747,7 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
         }
       }
     }
-    ;
+
     final ExecutorService executor = ExecutorUtil.newMDCAwareCachedThreadPool("batchIndexing");
     final int numThreads = random().nextInt(TEST_NIGHTLY ? 4 : 2) + 1;
     final List<Future<?>> futures = new ArrayList<>(numThreads);
@@ -701,12 +768,13 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
     assertEquals(
         totalDocsExpected, cloudClient.query(params("q", "*:*")).getResults().getNumFound());
     checkShardConsistency(
+        collectionName,
         params("q", "*:*", "rows", "" + totalDocsExpected, "_trace", "batches_done"));
   }
 
   public void testConcurrentIndexing() throws Exception {
     final CloudSolrClient cloudClient = cluster.getSolrClient();
-    final String collectionName = createAndSetNewDefaultCollection();
+    final String collectionName = createNewCollection();
 
     final int numDocs = atLeast(50);
     final JettySolrRunner nodeToUpdate = cluster.getRandomJetty(random());
@@ -721,9 +789,12 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
       indexClient.blockUntilFinished();
 
       assertEquals(0, indexClient.commit().getStatus());
-      assertEquals(numDocs, cloudClient.query(params("q", "*:*")).getResults().getNumFound());
+      assertEquals(
+          numDocs,
+          cloudClient.query(collectionName, params("q", "*:*")).getResults().getNumFound());
 
-      checkShardConsistency(params("q", "*:*", "rows", "" + (1 + numDocs), "_trace", "addAll"));
+      checkShardConsistency(
+          collectionName, params("q", "*:*", "rows", "" + (1 + numDocs), "_trace", "addAll"));
     }
   }
 
@@ -735,16 +806,14 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
    * @see #cluster
    * @see CloudInspectUtil#showDiff
    */
-  private void checkShardConsistency(final SolrParams params) throws Exception {
+  private void checkShardConsistency(String collectionName, final SolrParams params)
+      throws Exception {
     // TODO: refactor into static in CloudInspectUtil w/ DocCollection param?
     // TODO: refactor to take in a BiFunction<QueryResponse,QueryResponse,Boolean> ?
 
     final SolrParams perReplicaParams = SolrParams.wrapDefaults(params("distrib", "false"), params);
     final DocCollection collection =
-        cluster
-            .getSolrClient()
-            .getClusterState()
-            .getCollection(cluster.getSolrClient().getDefaultCollection());
+        cluster.getSolrClient().getClusterState().getCollection(collectionName);
     log.info("Checking shard consistency via: {}", perReplicaParams);
     for (Map.Entry<String, Slice> entry : collection.getActiveSlicesMap().entrySet()) {
       final String shardName = entry.getKey();
diff --git a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionOnCommitTest.java b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionOnCommitTest.java
index c3eb5f9f338..a4c39ea7c7d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionOnCommitTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionOnCommitTest.java
@@ -74,7 +74,6 @@ public class HttpPartitionOnCommitTest extends BasicDistributedZkTest {
     // create a collection that has 2 shard and 2 replicas
     String testCollectionName = "c8n_2x2_commits";
     createCollection(testCollectionName, "conf1", 2, 2);
-    cloudClient.setDefaultCollection(testCollectionName);
 
     List<Replica> notLeaders = ensureAllReplicasAreActive(testCollectionName, "shard1", 2, 2, 30);
     assertEquals(
@@ -129,7 +128,6 @@ public class HttpPartitionOnCommitTest extends BasicDistributedZkTest {
     // create a collection that has 1 shard and 3 replicas
     String testCollectionName = "c8n_1x3_commits";
     createCollection(testCollectionName, "conf1", 1, 3);
-    cloudClient.setDefaultCollection(testCollectionName);
 
     List<Replica> notLeaders = ensureAllReplicasAreActive(testCollectionName, "shard1", 1, 3, 30);
     assertEquals(
diff --git a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
index 0aba6331179..b14ec3dd5ae 100644
--- a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
@@ -24,12 +24,10 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import org.apache.lucene.tests.util.LuceneTestCase;
@@ -38,8 +36,6 @@ import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.cloud.SocketProxy;
-import org.apache.solr.client.solrj.impl.CloudLegacySolrClient;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
@@ -98,20 +94,6 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     fixShardCount(3);
   }
 
-  /** We need to turn off directUpdatesToLeadersOnly due to SOLR-9512 */
-  @Override
-  protected CloudSolrClient createCloudClient(String defaultCollection) {
-    var client =
-        new CloudLegacySolrClient.Builder(
-                Collections.singletonList(zkServer.getZkAddress()), Optional.empty())
-            .sendDirectUpdatesToAnyShardReplica()
-            .withConnectionTimeout(5000, TimeUnit.MILLISECONDS)
-            .withSocketTimeout(10000, TimeUnit.MILLISECONDS)
-            .build();
-    if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
-    return client;
-  }
-
   /** Overrides the parent implementation to install a SocketProxy in-front of the Jetty server. */
   @Override
   public JettySolrRunner createJetty(
@@ -161,9 +143,8 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
       TestInjection.prepRecoveryOpPauseForever = "true:100";
 
       createCollection(testCollectionName, "conf1", 1, 2);
-      cloudClient.setDefaultCollection(testCollectionName);
 
-      sendDoc(1);
+      sendDoc(testCollectionName, 1);
 
       JettySolrRunner leaderJetty =
           getJettyOnPort(getReplicaPort(getShardLeader(testCollectionName, "shard1", 1000)));
@@ -179,7 +160,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
       leaderProxy.close();
 
       // indexing during a partition
-      int achievedRf = sendDoc(2, leaderJetty);
+      int achievedRf = sendDoc(testCollectionName, 2, leaderJetty);
       assertEquals("Unexpected achieved replication factor", 1, achievedRf);
       try (ZkShardTerms zkShardTerms =
           new ZkShardTerms(
@@ -219,9 +200,8 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     // create a collection that has 1 shard but 2 replicas
     String testCollectionName = "c8n_1x2";
     createCollectionRetry(testCollectionName, "conf1", 1, 2);
-    cloudClient.setDefaultCollection(testCollectionName);
 
-    sendDoc(1);
+    sendDoc(testCollectionName, 1);
 
     Replica notLeader =
         ensureAllReplicasAreActive(testCollectionName, "shard1", 1, 2, maxWaitSecsToSeeAllActive)
@@ -238,7 +218,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     leaderProxy.close();
 
     // indexing during a partition
-    sendDoc(2, leaderJetty);
+    sendDoc(testCollectionName, 2, leaderJetty);
     // replica should publish itself as DOWN if the network is not healed after some amount time
     waitForState(testCollectionName, notLeader.getName(), DOWN, 10000);
 
@@ -248,7 +228,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     List<Replica> notLeaders =
         ensureAllReplicasAreActive(testCollectionName, "shard1", 1, 2, maxWaitSecsToSeeAllActive);
 
-    int achievedRf = sendDoc(3);
+    int achievedRf = sendDoc(testCollectionName, 3);
     if (achievedRf == 1) {
       // this case can happen when leader reuse an connection get established before network
       // partition
@@ -294,7 +274,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
         }
       }
       // always send doc directly to leader without going through proxy
-      sendDoc(d + 4, leaderJetty); // 4 is offset as we've already indexed 1-3
+      sendDoc(testCollectionName, d + 4, leaderJetty); // 4 is offset as we've already indexed 1-3
     }
 
     // restore connectivity if lost
@@ -358,8 +338,6 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     String testCollectionName = "c8n_1x3";
     createCollectionRetry(testCollectionName, "conf1", 1, 3);
 
-    cloudClient.setDefaultCollection(testCollectionName);
-
     sendDoc(1);
 
     List<Replica> notLeaders =
@@ -385,7 +363,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     leaderProxy.close();
 
     // indexing during a partition
-    sendDoc(2, leaderJetty);
+    sendDoc(testCollectionName, 2, leaderJetty);
 
     Thread.sleep(sleepMsBeforeHealPartition);
     proxy0.reopen();
@@ -393,7 +371,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     SocketProxy proxy1 = getProxyForReplica(notLeaders.get(1));
     proxy1.close();
 
-    sendDoc(3, leaderJetty);
+    sendDoc(testCollectionName, 3, leaderJetty);
 
     Thread.sleep(sleepMsBeforeHealPartition);
     proxy1.reopen();
@@ -420,9 +398,8 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
 
     String testCollectionName = "c8n_1x2_leader_session_loss";
     createCollectionRetry(testCollectionName, "conf1", 1, 2);
-    cloudClient.setDefaultCollection(testCollectionName);
 
-    sendDoc(1);
+    sendDoc(testCollectionName, 1);
 
     List<Replica> notLeaders =
         ensureAllReplicasAreActive(testCollectionName, "shard1", 1, 2, maxWaitSecsToSeeAllActive);
@@ -563,18 +540,18 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     }
   }
 
-  protected SolrClient getHttpSolrClient(Replica replica, String coll) {
+  protected SolrClient getHttpSolrClient(Replica replica, String collection) {
     ZkCoreNodeProps zkProps = new ZkCoreNodeProps(replica);
-    String url = zkProps.getBaseUrl() + "/" + coll;
+    String url = zkProps.getBaseUrl() + "/" + collection;
     return getHttpSolrClient(url);
   }
 
   // Send doc directly to a server (without going through proxy)
-  protected int sendDoc(int docId, JettySolrRunner leaderJetty)
+  protected int sendDoc(String collectionName, int docId, JettySolrRunner leaderJetty)
       throws IOException, SolrServerException {
     try (SolrClient solrClient =
         new HttpSolrClient.Builder(leaderJetty.getBaseUrl().toString()).build()) {
-      return sendDoc(docId, solrClient, cloudClient.getDefaultCollection());
+      return sendDoc(docId, solrClient, collectionName);
     }
   }
 
@@ -582,6 +559,10 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     return sendDoc(docId, cloudClient, cloudClient.getDefaultCollection());
   }
 
+  protected int sendDoc(String collectionName, int docId) throws Exception {
+    return sendDoc(docId, cloudClient, collectionName);
+  }
+
   protected int sendDoc(int docId, SolrClient solrClient, String collection)
       throws IOException, SolrServerException {
     SolrInputDocument doc = new SolrInputDocument();
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java
index b67d90f4763..5011ba3abe6 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java
@@ -61,9 +61,8 @@ public class LeaderFailoverAfterPartitionTest extends HttpPartitionTest {
     // create a collection that has 1 shard but 3 replicas
     String testCollectionName = "c8n_1x3_lf"; // _lf is leader fails
     createCollection(testCollectionName, "conf1", 1, 3);
-    cloudClient.setDefaultCollection(testCollectionName);
 
-    sendDoc(1);
+    sendDoc(testCollectionName, 1);
 
     List<Replica> notLeaders =
         ensureAllReplicasAreActive(testCollectionName, "shard1", 1, 3, maxWaitSecsToSeeAllActive);
@@ -78,13 +77,12 @@ public class LeaderFailoverAfterPartitionTest extends HttpPartitionTest {
         notLeaders.size());
 
     // ok, now introduce a network partition between the leader and the replica
-    SocketProxy proxy0 = null;
-    proxy0 = getProxyForReplica(notLeaders.get(0));
+    SocketProxy proxy0 = getProxyForReplica(notLeaders.get(0));
 
     proxy0.close();
 
     // indexing during a partition
-    sendDoc(2);
+    sendDoc(testCollectionName, 2);
 
     Thread.sleep(sleepMsBeforeHealPartition);
 
@@ -94,7 +92,7 @@ public class LeaderFailoverAfterPartitionTest extends HttpPartitionTest {
 
     proxy1.close();
 
-    sendDoc(3);
+    sendDoc(testCollectionName, 3);
 
     Thread.sleep(sleepMsBeforeHealPartition);
     proxy1.reopen();
@@ -103,7 +101,7 @@ public class LeaderFailoverAfterPartitionTest extends HttpPartitionTest {
     notLeaders =
         ensureAllReplicasAreActive(testCollectionName, "shard1", 1, 3, maxWaitSecsToSeeAllActive);
 
-    sendDoc(4);
+    sendDoc(testCollectionName, 4);
 
     assertDocsExistInAllReplicas(notLeaders, testCollectionName, 1, 4);
 
@@ -125,7 +123,7 @@ public class LeaderFailoverAfterPartitionTest extends HttpPartitionTest {
 
     // indexing during a partition
     // doc should be on leader and 1 replica
-    sendDoc(5);
+    sendDoc(testCollectionName, 5);
 
     try (SolrClient server = getHttpSolrClient(leader, testCollectionName)) {
       assertDocExists(server, "5");
@@ -192,7 +190,7 @@ public class LeaderFailoverAfterPartitionTest extends HttpPartitionTest {
     SolrInputDocument doc = new SolrInputDocument();
     doc.addField(id, String.valueOf(6));
     doc.addField("a_t", "hello" + 6);
-    sendDocsWithRetry(Collections.singletonList(doc), 1, 3, 1);
+    sendDocsWithRetry(testCollectionName, Collections.singletonList(doc), 1, 3, 1);
 
     Set<String> replicasToCheck = new HashSet<>();
     for (Replica stillUp : participatingReplicas) replicasToCheck.add(stillUp.getName());
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java
index 70b98586ddb..b2c3e6eab6d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java
@@ -65,7 +65,6 @@ public class LeaderTragicEventTest extends SolrCloudTestCase {
   public void setUp() throws Exception {
     super.setUp();
     collection = getSaferTestName();
-    cluster.getSolrClient().setDefaultCollection(collection);
   }
 
   @Override
@@ -82,7 +81,7 @@ public class LeaderTragicEventTest extends SolrCloudTestCase {
     cluster.waitForActiveCollection(collection, 1, 2);
 
     UpdateResponse updateResponse =
-        new UpdateRequest().add("id", "1").commit(cluster.getSolrClient(), null);
+        new UpdateRequest().add("id", "1").commit(cluster.getSolrClient(), collection);
     assertEquals(0, updateResponse.getStatus());
 
     Replica oldLeader = corruptLeader(collection);
@@ -112,7 +111,7 @@ public class LeaderTragicEventTest extends SolrCloudTestCase {
         getNonLeader(shard).getNodeName());
 
     // Check that we can continue indexing after this
-    updateResponse = new UpdateRequest().add("id", "2").commit(cluster.getSolrClient(), null);
+    updateResponse = new UpdateRequest().add("id", "2").commit(cluster.getSolrClient(), collection);
     assertEquals(0, updateResponse.getStatus());
     try (SolrClient followerClient = new HttpSolrClient.Builder(oldLeader.getCoreUrl()).build()) {
       QueryResponse queryResponse = new QueryRequest(new SolrQuery("*:*")).process(followerClient);
diff --git a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
index c4b6a485406..7c733f3df47 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MigrateRouteKeyTest.java
@@ -103,9 +103,9 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
   @Test
   public void multipleShardMigrateTest() throws Exception {
 
-    CollectionAdminRequest.createCollection("sourceCollection", "conf", 2, 1)
+    String sourceCollection = "sourceCollection";
+    CollectionAdminRequest.createCollection(sourceCollection, "conf", 2, 1)
         .process(cluster.getSolrClient());
-    cluster.getSolrClient().setDefaultCollection("sourceCollection");
 
     final String splitKey = "a";
     final int BIT_SEP = 1;
@@ -120,7 +120,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
       SolrInputDocument doc = new SolrInputDocument();
       doc.addField("id", key + "!" + id);
       doc.addField("n_ti", id);
-      cluster.getSolrClient().add("sourceCollection", doc);
+      cluster.getSolrClient().add(sourceCollection, doc);
       if (splitKey.equals(shardKey)) splitKeyCount++;
     }
     assertTrue(splitKeyCount > 0);
@@ -129,7 +129,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
     CollectionAdminRequest.createCollection(targetCollection, "conf", 1, 1)
         .process(cluster.getSolrClient());
 
-    Indexer indexer = new Indexer(cluster.getSolrClient(), splitKey, 1, 30);
+    Indexer indexer = new Indexer(cluster.getSolrClient(), sourceCollection, splitKey, 1, 30);
     indexer.start();
 
     DocCollection state = getCollectionState(targetCollection);
@@ -144,7 +144,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
 
       invokeCollectionMigration(
           CollectionAdminRequest.migrateData(
-                  "sourceCollection", targetCollection, splitKey + "/" + BIT_SEP + "!")
+                  sourceCollection, targetCollection, splitKey + "/" + BIT_SEP + "!")
               .setForwardTimeout(45));
 
       long finishTime = System.nanoTime();
@@ -153,12 +153,12 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
       splitKeyCount += indexer.getSplitKeyCount();
 
       try {
-        cluster.getSolrClient().deleteById("a/" + BIT_SEP + "!104");
+        cluster.getSolrClient().deleteById(sourceCollection, "a/" + BIT_SEP + "!104");
         splitKeyCount--;
       } catch (Exception e) {
         log.warn("Error deleting document a/{}!104", BIT_SEP, e);
       }
-      cluster.getSolrClient().commit();
+      cluster.getSolrClient().commit(sourceCollection);
       collectionClient.commit();
 
       solrQuery = new SolrQuery("*:*").setRows(1000);
@@ -171,7 +171,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
 
       waitForState(
           "Expected to find routing rule for split key " + splitKey,
-          "sourceCollection",
+          sourceCollection,
           (n, c) -> {
             if (c == null) return false;
             Slice shard = c.getSlice("shard2");
@@ -181,7 +181,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
             return true;
           });
 
-      boolean ruleRemoved = waitForRuleToExpire("sourceCollection", "shard2", splitKey, finishTime);
+      boolean ruleRemoved = waitForRuleToExpire(sourceCollection, "shard2", splitKey, finishTime);
       assertTrue("Routing rule was not expired", ruleRemoved);
     }
   }
@@ -190,14 +190,17 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
     final int seconds;
     final CloudSolrClient cloudClient;
     final String splitKey;
+    private final String collection;
     int splitKeyCount = 0;
     final int bitSep;
 
-    public Indexer(CloudSolrClient cloudClient, String splitKey, int bitSep, int seconds) {
-      this.seconds = seconds;
+    public Indexer(
+        CloudSolrClient cloudClient, String collection, String splitKey, int bitSep, int seconds) {
       this.cloudClient = cloudClient;
+      this.collection = collection;
       this.splitKey = splitKey;
       this.bitSep = bitSep;
+      this.seconds = seconds;
     }
 
     @Override
@@ -210,7 +213,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
         doc.addField("id", shardKey + (bitSep != -1 ? "/" + bitSep : "") + "!" + id);
         doc.addField("n_ti", id);
         try {
-          cloudClient.add(doc);
+          cloudClient.add(collection, doc);
           if (splitKey.equals(shardKey)) splitKeyCount++;
         } catch (Exception e) {
           log.error("Exception while adding document id: {}", doc.getField("id"), e);
diff --git a/solr/core/src/test/org/apache/solr/cloud/MissingSegmentRecoveryTest.java b/solr/core/src/test/org/apache/solr/cloud/MissingSegmentRecoveryTest.java
index 0849543b959..22c2fb13288 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MissingSegmentRecoveryTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MissingSegmentRecoveryTest.java
@@ -55,7 +55,6 @@ public class MissingSegmentRecoveryTest extends SolrCloudTestCase {
         .process(cluster.getSolrClient());
     waitForState(
         "Expected a collection with one shard and two replicas", collection, clusterShape(1, 2));
-    cluster.getSolrClient().setDefaultCollection(collection);
 
     List<SolrInputDocument> docs = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
@@ -64,8 +63,8 @@ public class MissingSegmentRecoveryTest extends SolrCloudTestCase {
       docs.add(doc);
     }
 
-    cluster.getSolrClient().add(docs);
-    cluster.getSolrClient().commit();
+    cluster.getSolrClient().add(collection, docs);
+    cluster.getSolrClient().commit(collection);
 
     DocCollection state = getCollectionState(collection);
     leader = state.getLeader("shard1");
diff --git a/solr/core/src/test/org/apache/solr/cloud/NestedShardedAtomicUpdateTest.java b/solr/core/src/test/org/apache/solr/cloud/NestedShardedAtomicUpdateTest.java
index e99032c7520..c19ff114441 100644
--- a/solr/core/src/test/org/apache/solr/cloud/NestedShardedAtomicUpdateTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/NestedShardedAtomicUpdateTest.java
@@ -53,8 +53,7 @@ public class NestedShardedAtomicUpdateTest extends SolrCloudTestCase {
 
     configureCluster(1).addConfig(configName, configDir).configure();
 
-    cloudClient = cluster.getSolrClient();
-    cloudClient.setDefaultCollection(DEFAULT_COLLECTION);
+    cloudClient = cluster.getSolrClient(DEFAULT_COLLECTION);
 
     CollectionAdminRequest.createCollection(DEFAULT_COLLECTION, configName, 4, 1)
         .withProperty("config", "solrconfig-tlog.xml")
diff --git a/solr/core/src/test/org/apache/solr/cloud/ParallelCommitExecutionTest.java b/solr/core/src/test/org/apache/solr/cloud/ParallelCommitExecutionTest.java
index a786ea0bc05..5351f6627ba 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ParallelCommitExecutionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ParallelCommitExecutionTest.java
@@ -43,8 +43,8 @@ public class ParallelCommitExecutionTest extends SolrCloudTestCase {
   private static final String DEBUG_LABEL = MethodHandles.lookup().lookupClass().getName();
   private static final String COLLECTION_NAME = DEBUG_LABEL + "_collection";
 
-  /** A basic client for operations at the cloud level, default collection will be set */
-  private static CloudSolrClient CLOUD_CLIENT;
+  /** A collection specific client for operations at the cloud level */
+  private static CloudSolrClient COLLECTION_CLIENT;
 
   private static int expectCount;
 
@@ -73,16 +73,15 @@ public class ParallelCommitExecutionTest extends SolrCloudTestCase {
         .setProperties(collectionProperties)
         .process(cluster.getSolrClient());
 
-    CLOUD_CLIENT = cluster.getSolrClient();
-    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
-    waitForRecoveriesToFinish(CLOUD_CLIENT);
+    COLLECTION_CLIENT = cluster.getSolrClient(COLLECTION_NAME);
+    waitForRecoveriesToFinish(COLLECTION_CLIENT);
   }
 
   @AfterClass
   public static void afterClass() throws Exception {
-    if (null != CLOUD_CLIENT) {
-      CLOUD_CLIENT.close();
-      CLOUD_CLIENT = null;
+    if (null != COLLECTION_CLIENT) {
+      COLLECTION_CLIENT.close();
+      COLLECTION_CLIENT = null;
     }
   }
 
@@ -96,7 +95,7 @@ public class ParallelCommitExecutionTest extends SolrCloudTestCase {
   @Test
   public void testParallelOk() throws Exception {
     initSyncVars();
-    CLOUD_CLIENT.commit(true, true);
+    COLLECTION_CLIENT.commit(true, true);
     assertEquals(0, countdown.getCount());
     assertEquals(expectCount, countup.get());
   }
diff --git a/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java b/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
index 6f95e394fd9..cc751d69da4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
@@ -110,8 +110,6 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
 
     createCollectionWithRetry(testCollectionName, numShards, replicationFactor);
 
-    cloudClient.setDefaultCollection(testCollectionName);
-
     List<Replica> replicas =
         ensureAllReplicasAreActive(testCollectionName, shardId, numShards, replicationFactor, 30);
     assertEquals("Expected active 1 replicas for " + testCollectionName, 1, replicas.size());
@@ -222,7 +220,7 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
       int expectedRfByIds,
       Set<Integer> byQueriesSet,
       int expectedRfDBQ,
-      String coll)
+      String collectionName)
       throws Exception {
     // First add the docs indicated
     List<String> byIdsList = new ArrayList<>();
@@ -244,17 +242,17 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
     }
 
     // Add the docs.
-    sendDocsWithRetry(batch, expectedRfDBQ, 5, 1);
+    sendDocsWithRetry(collectionName, batch, expectedRfDBQ, 5, 1);
 
     // Delete the docs by ID indicated
     UpdateRequest req = new UpdateRequest();
     req.deleteById(byIdsList);
-    sendNonDirectUpdateRequestReplicaWithRetry(rep, req, expectedRfByIds, coll);
+    sendNonDirectUpdateRequestReplicaWithRetry(rep, req, expectedRfByIds, collectionName);
 
     // Delete the docs by query indicated.
     req = new UpdateRequest();
     req.deleteByQuery("id:(" + StringUtils.join(byQueryList, " OR ") + ")");
-    sendNonDirectUpdateRequestReplicaWithRetry(rep, req, expectedRfDBQ, coll);
+    sendNonDirectUpdateRequestReplicaWithRetry(rep, req, expectedRfDBQ, collectionName);
   }
 
   protected void sendNonDirectUpdateRequestReplicaWithRetry(
@@ -297,30 +295,30 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
     final int minRf = 2;
 
     createCollectionWithRetry(testCollectionName, numShards, replicationFactor);
-    cloudClient.setDefaultCollection(testCollectionName);
 
     List<Replica> replicas =
         ensureAllReplicasAreActive(testCollectionName, shardId, numShards, replicationFactor, 30);
     assertEquals("Expected 2 active replicas for " + testCollectionName, 2, replicas.size());
 
     log.info("Indexing docId=1");
-    int rf = sendDoc(1);
+    int rf = sendDoc(testCollectionName, 1);
     assertRf(3, "all replicas should be active", rf);
 
     // Uses cloudClient to do it's work
-    doDBIdWithRetry(3, 5, "deletes should have propagated to all 3 replicas", 1);
-    doDBQWithRetry(3, 5, "deletes should have propagated to all 3 replicas", 1);
+    doDBIdWithRetry(
+        testCollectionName, 3, 5, "deletes should have propagated to all 3 replicas", 1);
+    doDBQWithRetry(testCollectionName, 3, 5, "deletes should have propagated to all 3 replicas", 1);
 
     log.info("Closing one proxy port");
     getProxyForReplica(replicas.get(0)).close();
 
     log.info("Indexing docId=2");
-    rf = sendDoc(2);
+    rf = sendDoc(testCollectionName, 2);
     assertRf(2, "one replica should be down", rf);
 
     // Uses cloudClient to do it's work
-    doDBQWithRetry(2, 5, "deletes should have propagated to 2 replicas", 1);
-    doDBIdWithRetry(2, 5, "deletes should have propagated to 2 replicas", 1);
+    doDBQWithRetry(testCollectionName, 2, 5, "deletes should have propagated to 2 replicas", 1);
+    doDBIdWithRetry(testCollectionName, 2, 5, "deletes should have propagated to 2 replicas", 1);
 
     // SOLR-13599 sanity check if problem is related to sending a batch
     List<SolrInputDocument> batch = new ArrayList<>(15);
@@ -331,18 +329,19 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
       batch.add(doc);
     }
     log.info("Indexing batch of documents (30-45)");
-    int batchRf = sendDocsWithRetry(batch, minRf, 5, 1);
+    int batchRf = sendDocsWithRetry(testCollectionName, batch, minRf, 5, 1);
     assertRf(2, "batch should have succeeded, only one replica should be down", batchRf);
 
     log.info("Closing second proxy port");
     getProxyForReplica(replicas.get(1)).close();
 
     log.info("Indexing docId=3");
-    rf = sendDoc(3);
+    rf = sendDoc(testCollectionName, 3);
     assertRf(1, "both replicas should be down", rf);
 
-    doDBQWithRetry(1, 5, "deletes should have propagated to only 1 replica", 1);
-    doDBIdWithRetry(1, 5, "deletes should have propagated to only 1 replica", 1);
+    doDBQWithRetry(testCollectionName, 1, 5, "deletes should have propagated to only 1 replica", 1);
+    doDBIdWithRetry(
+        testCollectionName, 1, 5, "deletes should have propagated to only 1 replica", 1);
 
     // heal the partitions
     log.info("Re-opening closed proxy ports");
@@ -354,11 +353,11 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
     ensureAllReplicasAreActive(testCollectionName, shardId, numShards, replicationFactor, 30);
 
     log.info("Indexing docId=4");
-    rf = sendDoc(4);
+    rf = sendDoc(testCollectionName, 4);
     assertRf(3, "all replicas have been healed", rf);
 
-    doDBQWithRetry(3, 5, "delete should have propagated to all 3 replicas", 1);
-    doDBIdWithRetry(3, 5, "delete should have propagated to all 3 replicas", 1);
+    doDBQWithRetry(testCollectionName, 3, 5, "delete should have propagated to all 3 replicas", 1);
+    doDBIdWithRetry(testCollectionName, 3, 5, "delete should have propagated to all 3 replicas", 1);
 
     // now send a batch
     batch = new ArrayList<>(10);
@@ -370,11 +369,13 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
     }
 
     log.info("Indexing batch of documents (5-14)");
-    batchRf = sendDocsWithRetry(batch, minRf, 5, 1);
+    batchRf = sendDocsWithRetry(testCollectionName, batch, minRf, 5, 1);
     assertRf(3, "batch add should have succeeded on all replicas", batchRf);
 
-    doDBQWithRetry(3, 5, "batch deletes should have propagated to all 3 replica", 15);
-    doDBIdWithRetry(3, 5, "batch deletes should have propagated to all 3 replica", 15);
+    doDBQWithRetry(
+        testCollectionName, 3, 5, "batch deletes should have propagated to all 3 replica", 15);
+    doDBIdWithRetry(
+        testCollectionName, 3, 5, "batch deletes should have propagated to all 3 replica", 15);
 
     // add some chaos to the batch
     log.info("Closing one proxy port (again)");
@@ -383,7 +384,7 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
     // send a single doc (again)
     // SOLR-13599 sanity check if problem is related to "re-closing" a port on the proxy
     log.info("Indexing docId=5");
-    rf = sendDoc(5);
+    rf = sendDoc(testCollectionName, 5);
     assertRf(2, "doc should have succeeded, only one replica should be down", rf);
 
     // now send a batch (again)
@@ -395,11 +396,13 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
       batch.add(doc);
     }
     log.info("Indexing batch of documents (15-29)");
-    batchRf = sendDocsWithRetry(batch, minRf, 5, 1);
+    batchRf = sendDocsWithRetry(testCollectionName, batch, minRf, 5, 1);
     assertRf(2, "batch should have succeeded, only one replica should be down", batchRf);
 
-    doDBQWithRetry(2, 5, "deletes should have propagated to only 1 replica", 15);
-    doDBIdWithRetry(2, 5, "deletes should have propagated to only 1 replica", 15);
+    doDBQWithRetry(
+        testCollectionName, 2, 5, "deletes should have propagated to only 1 replica", 15);
+    doDBIdWithRetry(
+        testCollectionName, 2, 5, "deletes should have propagated to only 1 replica", 15);
 
     // close the 2nd replica, and send a 3rd batch with expected achieved rf=1
     log.info("Closing second proxy port (again)");
@@ -413,14 +416,16 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
       batch.add(doc);
     }
 
-    batchRf = sendDocsWithRetry(batch, minRf, 5, 1);
+    batchRf = sendDocsWithRetry(testCollectionName, batch, minRf, 5, 1);
     assertRf(
         1,
         "batch should have succeeded on the leader only (both replicas should be down)",
         batchRf);
 
-    doDBQWithRetry(1, 5, "deletes should have propagated to only 1 replica", 15);
-    doDBIdWithRetry(1, 5, "deletes should have propagated to only 1 replica", 15);
+    doDBQWithRetry(
+        testCollectionName, 1, 5, "deletes should have propagated to only 1 replica", 15);
+    doDBIdWithRetry(
+        testCollectionName, 1, 5, "deletes should have propagated to only 1 replica", 15);
 
     getProxyForReplica(replicas.get(0)).reopen();
     getProxyForReplica(replicas.get(1)).reopen();
@@ -429,11 +434,12 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
     ensureAllReplicasAreActive(testCollectionName, shardId, numShards, replicationFactor, 30);
   }
 
-  protected void addDocs(Set<Integer> docIds, int expectedRf, int retries) throws Exception {
+  protected void addDocs(String collection, Set<Integer> docIds, int expectedRf, int retries)
+      throws Exception {
 
     Integer[] idList = docIds.toArray(new Integer[docIds.size()]);
     if (idList.length == 1) {
-      sendDoc(idList[0]);
+      sendDoc(collection, idList[0]);
       return;
     }
     List<SolrInputDocument> batch = new ArrayList<>(10);
@@ -443,53 +449,53 @@ public class ReplicationFactorTest extends AbstractFullDistribZkTestBase {
       doc.addField("a_t", "hello" + docId);
       batch.add(doc);
     }
-    sendDocsWithRetry(batch, expectedRf, retries, 1);
+    sendDocsWithRetry(collection, batch, expectedRf, retries, 1);
   }
 
-  protected void doDBQWithRetry(int expectedRf, int retries, String msg, int docsToAdd)
-      throws Exception {
+  protected void doDBQWithRetry(
+      String collection, int expectedRf, int retries, String msg, int docsToAdd) throws Exception {
     Set<Integer> docIds = getSomeIds(docsToAdd);
-    addDocs(docIds, expectedRf, retries);
+    addDocs(collection, docIds, expectedRf, retries);
     UpdateRequest req = new UpdateRequest();
     req.deleteByQuery("id:(" + StringUtils.join(docIds, " OR ") + ")");
-    doDelete(req, msg, expectedRf, retries);
+    doDelete(collection, req, msg, expectedRf, retries);
   }
 
-  protected void doDBIdWithRetry(int expectedRf, int retries, String msg, int docsToAdd)
-      throws Exception {
+  protected void doDBIdWithRetry(
+      String collection, int expectedRf, int retries, String msg, int docsToAdd) throws Exception {
     Set<Integer> docIds = getSomeIds(docsToAdd);
-    addDocs(docIds, expectedRf, retries);
+    addDocs(collection, docIds, expectedRf, retries);
     UpdateRequest req = new UpdateRequest();
     req.deleteById(StringUtils.join(docIds, ","));
-    doDelete(req, msg, expectedRf, retries);
+    doDelete(collection, req, msg, expectedRf, retries);
   }
 
-  protected void doDelete(UpdateRequest req, String msg, int expectedRf, int retries)
+  protected void doDelete(
+      String collection, UpdateRequest req, String msg, int expectedRf, int retries)
       throws IOException, SolrServerException, InterruptedException {
     int achievedRf = -1;
     for (int idx = 0; idx < retries; ++idx) {
-      NamedList<Object> response = cloudClient.request(req);
-      achievedRf =
-          cloudClient.getMinAchievedReplicationFactor(cloudClient.getDefaultCollection(), response);
+      NamedList<Object> response = cloudClient.request(req, collection);
+      achievedRf = cloudClient.getMinAchievedReplicationFactor(collection, response);
       if (achievedRf == expectedRf) return;
       Thread.sleep(1000);
     }
     assertEquals(msg, expectedRf, achievedRf);
   }
 
-  protected int sendDoc(int docId) throws Exception {
+  protected int sendDoc(String collectionName, int docId) throws Exception {
     UpdateRequest up = new UpdateRequest();
     SolrInputDocument doc = new SolrInputDocument();
     doc.addField(id, String.valueOf(docId));
     doc.addField("a_t", "hello" + docId);
     up.add(doc);
-    return runAndGetAchievedRf(up);
+    return runAndGetAchievedRf(collectionName, up);
   }
 
-  private int runAndGetAchievedRf(UpdateRequest up) throws SolrServerException, IOException {
-    NamedList<Object> response = cloudClient.request(up);
-    return cloudClient.getMinAchievedReplicationFactor(
-        cloudClient.getDefaultCollection(), response);
+  private int runAndGetAchievedRf(String collectionName, UpdateRequest up)
+      throws SolrServerException, IOException {
+    NamedList<Object> response = cloudClient.request(up, collectionName);
+    return cloudClient.getMinAchievedReplicationFactor(collectionName, response);
   }
 
   protected void assertRf(int expected, String explain, int actual) throws Exception {
diff --git a/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java b/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java
index 6166cb31224..d4822d718e3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java
@@ -58,7 +58,11 @@ class SegmentTerminateEarlyTestState {
   }
 
   void addDocuments(
-      CloudSolrClient cloudSolrClient, int numCommits, int numDocsPerCommit, boolean optimize)
+      String collection,
+      CloudSolrClient cloudSolrClient,
+      int numCommits,
+      int numDocsPerCommit,
+      boolean optimize)
       throws Exception {
     for (int cc = 1; cc <= numCommits; ++cc) {
       for (int nn = 1; nn <= numDocsPerCommit; ++nn) {
@@ -84,16 +88,17 @@ class SegmentTerminateEarlyTestState {
         doc.setField(TIMESTAMP_FIELD, MM);
         doc.setField(ODD_FIELD, "" + (numDocs % 2));
         doc.setField(QUAD_FIELD, "" + (numDocs % 4) + 1);
-        cloudSolrClient.add(doc);
+        cloudSolrClient.add(collection, doc);
       }
-      cloudSolrClient.commit();
+      cloudSolrClient.commit(collection);
     }
     if (optimize) {
-      cloudSolrClient.optimize();
+      cloudSolrClient.optimize(collection);
     }
   }
 
-  void queryTimestampDescending(CloudSolrClient cloudSolrClient) throws Exception {
+  void queryTimestampDescending(String collection, CloudSolrClient cloudSolrClient)
+      throws Exception {
     TestSegmentSorting.assertFalse(maxTimestampDocKeys.isEmpty());
     TestSegmentSorting.assertEquals("numDocs=" + numDocs + " is not even", 0, (numDocs % 2));
     final Long oddFieldValue = (long) (maxTimestampDocKeys.iterator().next() % 2);
@@ -102,7 +107,7 @@ class SegmentTerminateEarlyTestState {
     query.setFields(KEY_FIELD, ODD_FIELD, TIMESTAMP_FIELD);
     query.setRows(1);
     // CommonParams.SEGMENT_TERMINATE_EARLY parameter intentionally absent
-    final QueryResponse rsp = cloudSolrClient.query(query);
+    final QueryResponse rsp = cloudSolrClient.query(collection, query);
     // check correctness of the results count
     TestSegmentSorting.assertEquals("numFound", numDocs / 2, rsp.getResults().getNumFound());
     // check correctness of the first result
@@ -130,7 +135,8 @@ class SegmentTerminateEarlyTestState {
   }
 
   void queryTimestampDescendingSegmentTerminateEarlyYes(
-      CloudSolrClient cloudSolrClient, boolean appendKeyDescendingToSort) throws Exception {
+      String collection, CloudSolrClient cloudSolrClient, boolean appendKeyDescendingToSort)
+      throws Exception {
     TestSegmentSorting.assertFalse(maxTimestampDocKeys.isEmpty());
     TestSegmentSorting.assertEquals("numDocs=" + numDocs + " is not even", 0, (numDocs % 2));
     final Long oddFieldValue = (long) (maxTimestampDocKeys.iterator().next() % 2);
@@ -145,7 +151,7 @@ class SegmentTerminateEarlyTestState {
       query.set(ShardParams.SHARDS_INFO, shardsInfoWanted);
     }
     query.set(CommonParams.SEGMENT_TERMINATE_EARLY, true);
-    final QueryResponse rsp = cloudSolrClient.query(query);
+    final QueryResponse rsp = cloudSolrClient.query(collection, query);
     // check correctness of the results count
     TestSegmentSorting.assertTrue("numFound", rowsWanted <= rsp.getResults().getNumFound());
     TestSegmentSorting.assertTrue("numFound", rsp.getResults().getNumFound() <= numDocs / 2);
@@ -200,7 +206,8 @@ class SegmentTerminateEarlyTestState {
   }
 
   void queryTimestampDescendingSegmentTerminateEarlyNo(
-      CloudSolrClient cloudSolrClient, boolean appendKeyDescendingToSort) throws Exception {
+      String collection, CloudSolrClient cloudSolrClient, boolean appendKeyDescendingToSort)
+      throws Exception {
     TestSegmentSorting.assertFalse(maxTimestampDocKeys.isEmpty());
     TestSegmentSorting.assertEquals("numDocs=" + numDocs + " is not even", 0, (numDocs % 2));
     final Long oddFieldValue = (long) (maxTimestampDocKeys.iterator().next() % 2);
@@ -214,7 +221,7 @@ class SegmentTerminateEarlyTestState {
       query.set(ShardParams.SHARDS_INFO, shardsInfoWanted);
     }
     query.set(CommonParams.SEGMENT_TERMINATE_EARLY, false);
-    final QueryResponse rsp = cloudSolrClient.query(query);
+    final QueryResponse rsp = cloudSolrClient.query(collection, query);
     // check correctness of the results count
     TestSegmentSorting.assertEquals("numFound", numDocs / 2, rsp.getResults().getNumFound());
     // check correctness of the first result
@@ -266,7 +273,8 @@ class SegmentTerminateEarlyTestState {
   }
 
   void queryTimestampDescendingSegmentTerminateEarlyYesGrouped(
-      CloudSolrClient cloudSolrClient, boolean appendKeyDescendingToSort) throws Exception {
+      String collection, CloudSolrClient cloudSolrClient, boolean appendKeyDescendingToSort)
+      throws Exception {
     TestSegmentSorting.assertFalse(maxTimestampDocKeys.isEmpty());
     TestSegmentSorting.assertEquals("numDocs=" + numDocs + " is not even", 0, (numDocs % 2));
     final Long oddFieldValue = (long) (maxTimestampDocKeys.iterator().next() % 2);
@@ -279,7 +287,7 @@ class SegmentTerminateEarlyTestState {
     TestSegmentSorting.assertEquals("numDocs=" + numDocs + " is not quad-able", 0, (numDocs % 4));
     query.add("group.field", QUAD_FIELD);
     query.set("group", true);
-    final QueryResponse rsp = cloudSolrClient.query(query);
+    final QueryResponse rsp = cloudSolrClient.query(collection, query);
     // check correctness of the results count
     TestSegmentSorting.assertEquals(
         "matches", numDocs / 2, rsp.getGroupResponse().getValues().get(0).getMatches());
@@ -311,7 +319,8 @@ class SegmentTerminateEarlyTestState {
   }
 
   void queryTimestampAscendingSegmentTerminateEarlyYes(
-      CloudSolrClient cloudSolrClient, boolean appendKeyDescendingToSort) throws Exception {
+      String collection, CloudSolrClient cloudSolrClient, boolean appendKeyDescendingToSort)
+      throws Exception {
     TestSegmentSorting.assertFalse(minTimestampDocKeys.isEmpty());
     TestSegmentSorting.assertEquals("numDocs=" + numDocs + " is not even", 0, (numDocs % 2));
     final Long oddFieldValue = (long) (minTimestampDocKeys.iterator().next() % 2);
@@ -322,7 +331,7 @@ class SegmentTerminateEarlyTestState {
     query.setFields(KEY_FIELD, ODD_FIELD, TIMESTAMP_FIELD);
     query.setRows(1);
     query.set(CommonParams.SEGMENT_TERMINATE_EARLY, true);
-    final QueryResponse rsp = cloudSolrClient.query(query);
+    final QueryResponse rsp = cloudSolrClient.query(collection, query);
     // check correctness of the results count
     TestSegmentSorting.assertEquals("numFound", numDocs / 2, rsp.getResults().getNumFound());
     // check correctness of the first result
diff --git a/solr/core/src/test/org/apache/solr/cloud/SolrCloudExampleTest.java b/solr/core/src/test/org/apache/solr/cloud/SolrCloudExampleTest.java
index 0a688acabbf..3cd00c1e22d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SolrCloudExampleTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SolrCloudExampleTest.java
@@ -121,7 +121,6 @@ public class SolrCloudExampleTest extends AbstractFullDistribZkTestBase {
     // verify the collection is usable ...
     ensureAllReplicasAreActive(testCollectionName, "shard1", 2, 2, 20);
     ensureAllReplicasAreActive(testCollectionName, "shard2", 2, 2, 10);
-    cloudClient.setDefaultCollection(testCollectionName);
 
     int invalidToolExitStatus = 1;
     assertEquals(
@@ -163,15 +162,16 @@ public class SolrCloudExampleTest extends AbstractFullDistribZkTestBase {
       if (log.isInfoEnabled()) {
         log.info("POSTing {}", xml.toAbsolutePath());
       }
-      cloudClient.request(new StreamingUpdateRequest("/update", xml, "application/xml"));
+      cloudClient.request(
+          new StreamingUpdateRequest("/update", xml, "application/xml"), testCollectionName);
     }
-    cloudClient.commit();
+    cloudClient.commit(testCollectionName);
 
     int numFound = 0;
 
     // give the update a chance to take effect.
     for (int idx = 0; idx < 100; ++idx) {
-      QueryResponse qr = cloudClient.query(new SolrQuery("*:*"));
+      QueryResponse qr = cloudClient.query(testCollectionName, new SolrQuery("*:*"));
       numFound = (int) qr.getResults().getNumFound();
       if (numFound == expectedXmlDocCount) break;
       Thread.sleep(100);
diff --git a/solr/core/src/test/org/apache/solr/cloud/SplitShardTest.java b/solr/core/src/test/org/apache/solr/cloud/SplitShardTest.java
index dd20903deed..1e93dade6bc 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SplitShardTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SplitShardTest.java
@@ -176,20 +176,17 @@ public class SplitShardTest extends SolrCloudTestCase {
     assertEquals("wrong range in s1_1", expected1, delta1);
   }
 
-  CloudSolrClient createCollection(String collectionName, int repFactor) throws Exception {
+  private CloudSolrClient createCollection(String collectionName, int repFactor) throws Exception {
 
     CollectionAdminRequest.createCollection(collectionName, "conf", 1, repFactor)
         .process(cluster.getSolrClient());
 
     cluster.waitForActiveCollection(collectionName, 1, repFactor);
 
-    CloudSolrClient client = cluster.getSolrClient();
-    client.setDefaultCollection(collectionName);
-    return client;
+    return cluster.getSolrClient();
   }
 
-  long getNumDocs(CloudSolrClient client) throws Exception {
-    String collectionName = client.getDefaultCollection();
+  long getNumDocs(CloudSolrClient client, String collectionName) throws Exception {
     DocCollection collection = client.getClusterState().getCollection(collectionName);
     Collection<Slice> slices = collection.getSlices();
 
@@ -200,7 +197,7 @@ public class SplitShardTest extends SolrCloudTestCase {
       for (Replica replica : slice.getReplicas()) {
         SolrClient replicaClient =
             getHttpSolrClient(replica.getBaseUrl() + "/" + replica.getCoreName());
-        long numFound = 0;
+        long numFound;
         try {
           numFound =
               replicaClient
@@ -219,7 +216,8 @@ public class SplitShardTest extends SolrCloudTestCase {
       totCount += lastReplicaCount;
     }
 
-    long cloudClientDocs = client.query(new SolrQuery("*:*")).getResults().getNumFound();
+    long cloudClientDocs =
+        client.query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
     assertEquals(
         "Sum of shard count should equal distrib query doc count", totCount, cloudClientDocs);
     return totCount;
@@ -295,9 +293,9 @@ public class SplitShardTest extends SolrCloudTestCase {
       }
     }
 
-    client.commit(); // final commit is needed for visibility
+    client.commit(collectionName); // final commit is needed for visibility
 
-    long numDocs = getNumDocs(client);
+    long numDocs = getNumDocs(client, collectionName);
     if (numDocs != model.size()) {
       SolrDocumentList results =
           client
@@ -374,13 +372,13 @@ public class SplitShardTest extends SolrCloudTestCase {
 
   @Test
   public void testLiveSplit() throws Exception {
-    // Debugging tips: if this fails, it may be easier to debug by lowering the number fo threads to
+    // Debugging tips: if this fails, it may be easier to debug by lowering the number of threads to
     // 1 and looping the test until you get another failure. You may need to further instrument
     // things like DistributedZkUpdateProcessor to display the cluster state for the collection,
-    // etc. Using more threads increases the chance to hit a concurrency bug, but too many threads
-    // can overwhelm single-threaded buffering replay after the low level index split and result in
-    // subShard leaders that can't catch up and become active (a known issue that still needs to be
-    // resolved.)
+    // etc. Using more threads increases the chance of hitting a concurrency bug, but too many
+    // threads can overwhelm single-threaded buffering replay after the low level index split and
+    // result in subShard leaders that can't catch up and become active (a known issue that still
+    // needs to be resolved.)
     doLiveSplitShard("livesplit1", 1, 4);
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
index 2e531a7530d..90994febb24 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
@@ -49,8 +49,8 @@ public class TestCloudDeleteByQuery extends SolrCloudTestCase {
 
   private static final String COLLECTION_NAME = "test_col";
 
-  /** A basic client for operations at the cloud level, default collection will be set */
-  private static CloudSolrClient CLOUD_CLIENT;
+  /** A collection specific client for operations at the cloud level */
+  private static CloudSolrClient COLLECTION_CLIENT;
 
   /** A client for talking directly to the leader of shard1 */
   private static SolrClient S_ONE_LEADER_CLIENT;
@@ -75,9 +75,9 @@ public class TestCloudDeleteByQuery extends SolrCloudTestCase {
 
   @AfterClass
   public static void afterClass() throws Exception {
-    if (null != CLOUD_CLIENT) {
-      CLOUD_CLIENT.close();
-      CLOUD_CLIENT = null;
+    if (null != COLLECTION_CLIENT) {
+      COLLECTION_CLIENT.close();
+      COLLECTION_CLIENT = null;
     }
     if (null != S_ONE_LEADER_CLIENT) {
       S_ONE_LEADER_CLIENT.close();
@@ -119,8 +119,7 @@ public class TestCloudDeleteByQuery extends SolrCloudTestCase {
         .process(cluster.getSolrClient());
     cluster.waitForActiveCollection(COLLECTION_NAME, NUM_SHARDS, REPLICATION_FACTOR * NUM_SHARDS);
 
-    CLOUD_CLIENT = cluster.getSolrClient();
-    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    COLLECTION_CLIENT = cluster.getSolrClient(COLLECTION_NAME);
 
     ZkStateReader zkStateReader = cluster.getZkStateReader();
 
@@ -181,17 +180,17 @@ public class TestCloudDeleteByQuery extends SolrCloudTestCase {
     // routing
     assertEquals(
         0,
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .add(doc(f("id", S_ONE_PRE + random().nextInt()), f("expected_shard_s", "shard1")))
             .getStatus());
     assertEquals(
         0,
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .add(doc(f("id", S_TWO_PRE + random().nextInt()), f("expected_shard_s", "shard2")))
             .getStatus());
-    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
+    assertEquals(0, COLLECTION_CLIENT.commit().getStatus());
     SolrDocumentList docs =
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .query(
                 params(
                     "q", "*:*",
@@ -219,8 +218,8 @@ public class TestCloudDeleteByQuery extends SolrCloudTestCase {
 
   @Before
   public void clearCloudCollection() throws Exception {
-    assertEquals(0, CLOUD_CLIENT.deleteByQuery("*:*").getStatus());
-    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
+    assertEquals(0, COLLECTION_CLIENT.deleteByQuery("*:*").getStatus());
+    assertEquals(0, COLLECTION_CLIENT.commit().getStatus());
   }
 
   public void testMalformedDBQ(SolrClient client) {
@@ -235,7 +234,7 @@ public class TestCloudDeleteByQuery extends SolrCloudTestCase {
 
   //
   public void testMalformedDBQViaCloudClient() {
-    testMalformedDBQ(CLOUD_CLIENT);
+    testMalformedDBQ(COLLECTION_CLIENT);
   }
 
   public void testMalformedDBQViaShard1LeaderClient() {
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudPhrasesIdentificationComponent.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudPhrasesIdentificationComponent.java
index 5551f46f208..f0b4443e654 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudPhrasesIdentificationComponent.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudPhrasesIdentificationComponent.java
@@ -48,8 +48,8 @@ public class TestCloudPhrasesIdentificationComponent extends SolrCloudTestCase {
   private static final String DEBUG_LABEL = MethodHandles.lookup().lookupClass().getName();
   private static final String COLLECTION_NAME = DEBUG_LABEL + "_collection";
 
-  /** A basic client for operations at the cloud level, default collection will be set */
-  private static CloudSolrClient CLOUD_CLIENT;
+  /** A collection specific client for operations at the cloud level */
+  private static CloudSolrClient COLLECTION_CLIENT;
   /** One client per node */
   private static final ArrayList<SolrClient> CLIENTS = new ArrayList<>(5);
 
@@ -74,45 +74,44 @@ public class TestCloudPhrasesIdentificationComponent extends SolrCloudTestCase {
         .setProperties(collectionProperties)
         .process(cluster.getSolrClient());
 
-    CLOUD_CLIENT = cluster.getSolrClient();
-    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    COLLECTION_CLIENT = cluster.getSolrClient(COLLECTION_NAME);
 
-    waitForRecoveriesToFinish(CLOUD_CLIENT);
+    waitForRecoveriesToFinish(COLLECTION_CLIENT);
 
     for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       CLIENTS.add(getHttpSolrClient(jetty.getBaseUrl() + "/" + COLLECTION_NAME + "/"));
     }
 
     // index some docs...
-    CLOUD_CLIENT.add(
+    COLLECTION_CLIENT.add(
         sdoc(
             "id", "42",
             "title", "Tale of the Brown Fox: was he lazy?",
             "body", "No. The quick brown fox was a very brown fox who liked to get into trouble."));
-    CLOUD_CLIENT.add(
+    COLLECTION_CLIENT.add(
         sdoc(
             "id", "43",
             "title", "A fable in two acts",
             "body", "The brOwn fOx jumped. The lazy dog did not"));
-    CLOUD_CLIENT.add(
+    COLLECTION_CLIENT.add(
         sdoc(
             "id", "44",
             "title", "Why the LazY dog was lazy",
             "body",
                 "News flash: Lazy Dog was not actually lazy, it just seemed so compared to Fox"));
-    CLOUD_CLIENT.add(
+    COLLECTION_CLIENT.add(
         sdoc(
             "id", "45",
             "title", "Why Are We Lazy?",
             "body", "Because we are. that's why"));
-    CLOUD_CLIENT.commit();
+    COLLECTION_CLIENT.commit();
   }
 
   @AfterClass
   public static void afterClass() throws Exception {
-    if (null != CLOUD_CLIENT) {
-      CLOUD_CLIENT.close();
-      CLOUD_CLIENT = null;
+    if (null != COLLECTION_CLIENT) {
+      COLLECTION_CLIENT.close();
+      COLLECTION_CLIENT = null;
     }
     for (SolrClient client : CLIENTS) {
       client.close();
@@ -198,7 +197,7 @@ public class TestCloudPhrasesIdentificationComponent extends SolrCloudTestCase {
     int numClients = CLIENTS.size();
     int idx = TestUtil.nextInt(rand, 0, numClients);
 
-    return (idx == numClients) ? CLOUD_CLIENT : CLIENTS.get(idx);
+    return (idx == numClients) ? COLLECTION_CLIENT : CLIENTS.get(idx);
   }
 
   public static void waitForRecoveriesToFinish(CloudSolrClient client) throws Exception {
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudPseudoReturnFields.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudPseudoReturnFields.java
index c439cf9166c..3db1166fae0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudPseudoReturnFields.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudPseudoReturnFields.java
@@ -56,8 +56,8 @@ public class TestCloudPseudoReturnFields extends SolrCloudTestCase {
   private static final String DEBUG_LABEL = MethodHandles.lookup().lookupClass().getName();
   private static final String COLLECTION_NAME = DEBUG_LABEL + "_collection";
 
-  /** A basic client for operations at the cloud level, default collection will be set */
-  private static CloudSolrClient CLOUD_CLIENT;
+  /** A collection specific client for operations at the cloud level */
+  private static CloudSolrClient COLLECTION_CLIENT;
   /** One client per node */
   private static final ArrayList<SolrClient> CLIENTS = new ArrayList<>(5);
 
@@ -82,10 +82,9 @@ public class TestCloudPseudoReturnFields extends SolrCloudTestCase {
         .setProperties(collectionProperties)
         .process(cluster.getSolrClient());
 
-    CLOUD_CLIENT = cluster.getSolrClient();
-    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    COLLECTION_CLIENT = cluster.getSolrClient(COLLECTION_NAME);
 
-    waitForRecoveriesToFinish(CLOUD_CLIENT);
+    waitForRecoveriesToFinish(COLLECTION_CLIENT);
 
     for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       CLIENTS.add(getHttpSolrClient(jetty.getBaseUrl() + "/" + COLLECTION_NAME + "/"));
@@ -93,30 +92,30 @@ public class TestCloudPseudoReturnFields extends SolrCloudTestCase {
 
     assertEquals(
         0,
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .add(sdoc("id", "42", "newid", "420", "val_i", "1", "ssto", "X", "subject", "aaa"))
             .getStatus());
     assertEquals(
         0,
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .add(sdoc("id", "43", "newid", "430", "val_i", "9", "ssto", "X", "subject", "bbb"))
             .getStatus());
     assertEquals(
         0,
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .add(sdoc("id", "44", "newid", "440", "val_i", "4", "ssto", "X", "subject", "aaa"))
             .getStatus());
     assertEquals(
         0,
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .add(sdoc("id", "45", "newid", "450", "val_i", "6", "ssto", "X", "subject", "aaa"))
             .getStatus());
     assertEquals(
         0,
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .add(sdoc("id", "46", "newid", "460", "val_i", "3", "ssto", "X", "subject", "ggg"))
             .getStatus());
-    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
+    assertEquals(0, COLLECTION_CLIENT.commit().getStatus());
   }
 
   @Before
@@ -126,7 +125,7 @@ public class TestCloudPseudoReturnFields extends SolrCloudTestCase {
     // will get another copy of doc 99 in the ulog
     assertEquals(
         0,
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .add(
                 sdoc(
                     "id",
@@ -144,9 +143,9 @@ public class TestCloudPseudoReturnFields extends SolrCloudTestCase {
 
   @AfterClass
   public static void afterClass() throws Exception {
-    if (null != CLOUD_CLIENT) {
-      CLOUD_CLIENT.close();
-      CLOUD_CLIENT = null;
+    if (null != COLLECTION_CLIENT) {
+      COLLECTION_CLIENT.close();
+      COLLECTION_CLIENT = null;
     }
     for (SolrClient client : CLIENTS) {
       client.close();
@@ -169,7 +168,7 @@ public class TestCloudPseudoReturnFields extends SolrCloudTestCase {
                     params(
                         "includeDynamic", "true",
                         "showDefaults", "true"))
-                .process(CLOUD_CLIENT);
+                .process(COLLECTION_CLIENT);
         assertNotNull(
             "Test depends on a (dynamic) field matching '" + name + "', Null response", frsp);
         assertEquals(
@@ -1027,7 +1026,7 @@ public class TestCloudPseudoReturnFields extends SolrCloudTestCase {
   public static SolrClient getRandClient(Random rand) {
     int numClients = CLIENTS.size();
     int idx = TestUtil.nextInt(rand, 0, numClients);
-    return (idx == numClients) ? CLOUD_CLIENT : CLIENTS.get(idx);
+    return (idx == numClients) ? COLLECTION_CLIENT : CLIENTS.get(idx);
   }
 
   public static void waitForRecoveriesToFinish(CloudSolrClient client) throws Exception {
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudSearcherWarming.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudSearcherWarming.java
index 62ab06dd98e..99f61ce0c55 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudSearcherWarming.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudSearcherWarming.java
@@ -99,8 +99,6 @@ public class TestCloudSearcherWarming extends SolrCloudTestCase {
 
     cluster.waitForActiveCollection(collectionName, 1, 1);
 
-    solrClient.setDefaultCollection(collectionName);
-
     String addListenerCommand =
         "{"
             + "'add-listener' : {'name':'newSearcherListener','event':'newSearcher', 'class':'"
@@ -112,10 +110,10 @@ public class TestCloudSearcherWarming extends SolrCloudTestCase {
             + "}";
 
     ConfigRequest request = new ConfigRequest(addListenerCommand);
-    solrClient.request(request);
+    solrClient.request(request, collectionName);
 
-    solrClient.add(new SolrInputDocument("id", "1"));
-    solrClient.commit();
+    solrClient.add(collectionName, new SolrInputDocument("id", "1"));
+    solrClient.commit(collectionName);
 
     AtomicInteger expectedDocs = new AtomicInteger(1);
     AtomicReference<String> failingCoreNodeName = new AtomicReference<>();
@@ -158,8 +156,6 @@ public class TestCloudSearcherWarming extends SolrCloudTestCase {
     waitForState(
         "The collection should have 1 shard and 1 replica", collectionName, clusterShape(1, 1));
 
-    solrClient.setDefaultCollection(collectionName);
-
     String addListenerCommand =
         "{"
             + "'add-listener' : {'name':'newSearcherListener','event':'newSearcher', 'class':'"
@@ -171,15 +167,15 @@ public class TestCloudSearcherWarming extends SolrCloudTestCase {
             + "}";
 
     ConfigRequest request = new ConfigRequest(addListenerCommand);
-    solrClient.request(request);
+    solrClient.request(request, collectionName);
 
-    solrClient.add(new SolrInputDocument("id", "1"));
-    solrClient.commit();
+    solrClient.add(collectionName, new SolrInputDocument("id", "1"));
+    solrClient.commit(collectionName);
 
     AtomicInteger expectedDocs = new AtomicInteger(1);
     AtomicReference<String> failingCoreNodeName = new AtomicReference<>();
 
-    QueryResponse response = solrClient.query(new SolrQuery("*:*"));
+    QueryResponse response = solrClient.query(collectionName, new SolrQuery("*:*"));
     assertEquals(1, response.getResults().getNumFound());
 
     // reset
@@ -215,7 +211,7 @@ public class TestCloudSearcherWarming extends SolrCloudTestCase {
     // the newly created replica should become leader
     waitForState(
         "The collection should have 1 shard and 1 replica", collectionName, clusterShape(1, 1));
-    // the above call is not enough because we want to assert that the down'ed replica is not active
+    // the above call is not enough because we want to assert that the downed replica is not active
     // but clusterShape will also return true if replica is not live -- which we don't want
     CollectionStatePredicate collectionStatePredicate =
         (liveNodes, collectionState) -> {
@@ -241,7 +237,7 @@ public class TestCloudSearcherWarming extends SolrCloudTestCase {
     log.info("Starting old node 1");
     cluster.startJettySolrRunner(oldNode);
     waitForState("", collectionName, clusterShape(1, 2));
-    // invoke statewatcher explicitly to avoid race condition where the assert happens before the
+    // invoke statewatcher explicitly to avoid race condition where the assertion happens before the
     // state watcher is invoked by ZkStateReader
     ZkStateReader.from(solrClient).registerCollectionStateWatcher(collectionName, stateWatcher);
     assertNull(
@@ -259,7 +255,7 @@ public class TestCloudSearcherWarming extends SolrCloudTestCase {
     coreNameRef.set(null);
     coreNodeNameRef.set(null);
     failingCoreNodeName.set(null);
-    // has to be higher than the twice the recovery wait pause between attempts plus some margin
+    // has to be higher than twice the recovery wait pause between attempts plus some margin
     sleepTime.set(14000);
 
     // inject failure
@@ -268,7 +264,7 @@ public class TestCloudSearcherWarming extends SolrCloudTestCase {
     log.info("Starting old node 2");
     cluster.startJettySolrRunner(oldNode);
     waitForState("", collectionName, clusterShape(1, 2));
-    // invoke statewatcher explicitly to avoid race condition when the assert happens before the
+    // invoke statewatcher explicitly to avoid race condition when the assertion happens before the
     // state watcher is invoked by ZkStateReader
     ZkStateReader.from(solrClient).registerCollectionStateWatcher(collectionName, stateWatcher);
     assertNull(
@@ -370,7 +366,7 @@ public class TestCloudSearcherWarming extends SolrCloudTestCase {
         try {
           Thread.sleep(sleepTime.get());
         } catch (InterruptedException e) {
-          log.warn("newSearcher was interupdated", e);
+          log.warn("newSearcher was interrupted", e);
         }
         if (log.isInfoEnabled()) {
           log.info(
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestDynamicFieldNamesIndexCorrectly.java b/solr/core/src/test/org/apache/solr/cloud/TestDynamicFieldNamesIndexCorrectly.java
index 3ab47673e93..cd3061362dc 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestDynamicFieldNamesIndexCorrectly.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestDynamicFieldNamesIndexCorrectly.java
@@ -115,8 +115,7 @@ public class TestDynamicFieldNamesIndexCorrectly extends AbstractFullDistribZkTe
     final QueryResponse response;
     SolrDocumentList list = null;
     final QueryRequest req = new QueryRequest(solrQuery);
-    cloudClient.setDefaultCollection(collection);
-    response = req.process(cloudClient);
+    response = req.process(cloudClient, collection);
     list = response.getResults();
     return list;
   }
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionWithEmptyReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionWithEmptyReplica.java
index c6f64265de8..97df978bd10 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionWithEmptyReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionWithEmptyReplica.java
@@ -56,13 +56,12 @@ public class TestLeaderElectionWithEmptyReplica extends SolrCloudTestCase {
   @Test
   public void test() throws Exception {
     CloudSolrClient solrClient = cluster.getSolrClient();
-    solrClient.setDefaultCollection(COLLECTION_NAME);
     for (int i = 0; i < 10; i++) {
       SolrInputDocument doc = new SolrInputDocument();
       doc.addField("id", String.valueOf(i));
-      solrClient.add(doc);
+      solrClient.add(COLLECTION_NAME, doc);
     }
-    solrClient.commit();
+    solrClient.commit(COLLECTION_NAME);
 
     // find the leader node
     Replica replica = cluster.getZkStateReader().getLeaderRetry(COLLECTION_NAME, "shard1");
@@ -109,7 +108,7 @@ public class TestLeaderElectionWithEmptyReplica extends SolrCloudTestCase {
             .getSlice("shard1"));
 
     // sanity check that documents still exist
-    QueryResponse response = solrClient.query(new SolrQuery("*:*"));
+    QueryResponse response = solrClient.query(COLLECTION_NAME, new SolrQuery("*:*"));
     assertEquals("Indexed documents not found", 10, response.getResults().getNumFound());
   }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestOnReconnectListenerSupport.java b/solr/core/src/test/org/apache/solr/cloud/TestOnReconnectListenerSupport.java
index 0f89d8e0468..040fbc5e7eb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestOnReconnectListenerSupport.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestOnReconnectListenerSupport.java
@@ -64,7 +64,6 @@ public class TestOnReconnectListenerSupport extends AbstractFullDistribZkTestBas
     String testCollectionName = "c8n_onreconnect_1x1";
     String shardId = "shard1";
     createCollectionRetry(testCollectionName, "conf1", 1, 1);
-    cloudClient.setDefaultCollection(testCollectionName);
 
     Replica leader = getShardLeader(testCollectionName, shardId, 30 /* timeout secs */);
     JettySolrRunner leaderJetty = getJettyOnPort(getReplicaPort(leader));
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
index caca86ff72f..90baeb15644 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplicaErrorHandling.java
@@ -110,7 +110,6 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
     super.setUp();
     collectionName = suggestedCollectionName();
     expectThrows(SolrException.class, () -> getCollectionState(collectionName));
-    cluster.getSolrClient().setDefaultCollection(collectionName);
     cluster.waitForAllNodes(30);
   }
 
@@ -201,7 +200,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
           getHttpSolrClient(s.getReplicas(EnumSet.of(Replica.Type.PULL)).get(0).getCoreUrl())) {
         assertNumDocs(10, pullReplicaClient);
       }
-      assertNumDocs(10, cluster.getSolrClient());
+      assertNumDocs(10, cluster.getSolrClient(collectionName));
     } finally {
       log.info("Opening leader node");
       proxy.reopen();
@@ -271,6 +270,8 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
     long numFound = -1;
     while (!t.hasTimedOut()) {
       Thread.sleep(200);
+      // if client is an HttpSolrClient, then the collection is in the path
+      // otherwise with a CloudSolrClient then it's the defaultCollection
       numFound = client.query(new SolrQuery("*:*")).getResults().getNumFound();
       if (numFound == numDocs) {
         return;
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java b/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java
index e77556936c7..201747dbfd9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java
@@ -74,8 +74,8 @@ public class TestRandomFlRTGCloud extends SolrCloudTestCase {
   private static final String DEBUG_LABEL = MethodHandles.lookup().lookupClass().getName();
   private static final String COLLECTION_NAME = DEBUG_LABEL + "_collection";
 
-  /** A basic client for operations at the cloud level, default collection will be set */
-  private static CloudSolrClient CLOUD_CLIENT;
+  /** A collection specific client for operations at the cloud level */
+  private static CloudSolrClient COLLECTION_CLIENT;
   /** One client per node */
   private static final List<SolrClient> CLIENTS = Collections.synchronizedList(new ArrayList<>(5));
 
@@ -166,13 +166,12 @@ public class TestRandomFlRTGCloud extends SolrCloudTestCase {
 
     configureCluster(numNodes).addConfig(configName, configDir).configure();
 
-    CLOUD_CLIENT = cluster.getSolrClient();
-    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    COLLECTION_CLIENT = cluster.getSolrClient(COLLECTION_NAME);
 
     CollectionAdminRequest.createCollection(COLLECTION_NAME, configName, numShards, repFactor)
         .withProperty("config", "solrconfig-tlog.xml")
         .withProperty("schema", "schema-pseudo-fields.xml")
-        .process(CLOUD_CLIENT);
+        .process(COLLECTION_CLIENT);
 
     cluster.waitForActiveCollection(COLLECTION_NAME, numShards, repFactor * numShards);
 
@@ -183,9 +182,9 @@ public class TestRandomFlRTGCloud extends SolrCloudTestCase {
 
   @AfterClass
   public static void afterClass() throws Exception {
-    if (null != CLOUD_CLIENT) {
-      CLOUD_CLIENT.close();
-      CLOUD_CLIENT = null;
+    if (null != COLLECTION_CLIENT) {
+      COLLECTION_CLIENT.close();
+      COLLECTION_CLIENT = null;
     }
     for (SolrClient client : CLIENTS) {
       client.close();
@@ -641,7 +640,7 @@ public class TestRandomFlRTGCloud extends SolrCloudTestCase {
   public static SolrClient getRandClient(Random rand) {
     int numClients = CLIENTS.size();
     int idx = TestUtil.nextInt(rand, 0, numClients);
-    return (idx == numClients) ? CLOUD_CLIENT : CLIENTS.get(idx);
+    return (idx == numClients) ? COLLECTION_CLIENT : CLIENTS.get(idx);
   }
 
   public static void waitForRecoveriesToFinish(CloudSolrClient client) throws Exception {
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java b/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java
index efd22ac805e..5a4b24c897a 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java
@@ -42,6 +42,7 @@ public class TestSegmentSorting extends SolrCloudTestCase {
   private static final String configName = MethodHandles.lookup().lookupClass() + "_configSet";
 
   private static boolean compoundMergePolicySort = false;
+  private String collectionName;
 
   @BeforeClass
   public static void setupCluster() throws Exception {
@@ -60,7 +61,6 @@ public class TestSegmentSorting extends SolrCloudTestCase {
   @After
   public void ensureClusterEmpty() throws Exception {
     cluster.deleteAllCollections();
-    cluster.getSolrClient().setDefaultCollection(null);
     System.clearProperty("mergePolicySort");
     System.clearProperty("solr.tests.id.docValues");
   }
@@ -68,7 +68,7 @@ public class TestSegmentSorting extends SolrCloudTestCase {
   @Before
   public void createCollection() throws Exception {
 
-    final String collectionName = testName.getMethodName();
+    collectionName = testName.getMethodName();
     final CloudSolrClient cloudSolrClient = cluster.getSolrClient();
 
     final Map<String, String> collectionProperties = new HashMap<>();
@@ -87,8 +87,6 @@ public class TestSegmentSorting extends SolrCloudTestCase {
       assertEquals(RequestStatusState.COMPLETED, cmd.processAndWait(cloudSolrClient, 30));
     }
     cluster.waitForActiveCollection(collectionName, NUM_SHARDS, NUM_SHARDS * REPLICATION_FACTOR);
-
-    cloudSolrClient.setDefaultCollection(collectionName);
   }
 
   public void testSegmentTerminateEarly() throws Exception {
@@ -97,40 +95,40 @@ public class TestSegmentSorting extends SolrCloudTestCase {
     final CloudSolrClient cloudSolrClient = cluster.getSolrClient();
 
     // add some documents, then optimize to get merged-sorted segments
-    tstes.addDocuments(cloudSolrClient, 10, 10, true);
+    tstes.addDocuments(collectionName, cloudSolrClient, 10, 10, true);
 
     // CommonParams.SEGMENT_TERMINATE_EARLY parameter intentionally absent
-    tstes.queryTimestampDescending(cloudSolrClient);
+    tstes.queryTimestampDescending(collectionName, cloudSolrClient);
 
     // add a few more documents, but don't optimize to have some not-merge-sorted segments
-    tstes.addDocuments(cloudSolrClient, 2, 10, false);
+    tstes.addDocuments(collectionName, cloudSolrClient, 2, 10, false);
 
     // CommonParams.SEGMENT_TERMINATE_EARLY parameter now present
     tstes.queryTimestampDescendingSegmentTerminateEarlyYes(
-        cloudSolrClient, false /* appendKeyDescendingToSort */);
+        collectionName, cloudSolrClient, false /* appendKeyDescendingToSort */);
     tstes.queryTimestampDescendingSegmentTerminateEarlyNo(
-        cloudSolrClient, false /* appendKeyDescendingToSort */);
+        collectionName, cloudSolrClient, false /* appendKeyDescendingToSort */);
 
     // CommonParams.SEGMENT_TERMINATE_EARLY parameter present, but it won't be used
     tstes.queryTimestampDescendingSegmentTerminateEarlyYesGrouped(
-        cloudSolrClient, false /* appendKeyDescendingToSort */);
+        collectionName, cloudSolrClient, false /* appendKeyDescendingToSort */);
     // uses a sort order that is _not_ compatible with the merge sort order
     tstes.queryTimestampAscendingSegmentTerminateEarlyYes(
-        cloudSolrClient, false /* appendKeyDescendingToSort */);
+        collectionName, cloudSolrClient, false /* appendKeyDescendingToSort */);
 
     if (compoundMergePolicySort) {
       // CommonParams.SEGMENT_TERMINATE_EARLY parameter now present
       tstes.queryTimestampDescendingSegmentTerminateEarlyYes(
-          cloudSolrClient, true /* appendKeyDescendingToSort */);
+          collectionName, cloudSolrClient, true /* appendKeyDescendingToSort */);
       tstes.queryTimestampDescendingSegmentTerminateEarlyNo(
-          cloudSolrClient, true /* appendKeyDescendingToSort */);
+          collectionName, cloudSolrClient, true /* appendKeyDescendingToSort */);
 
-      // CommonParams.SEGMENT_TERMINATE_EARLY parameter present but it won't be used
+      // CommonParams.SEGMENT_TERMINATE_EARLY parameter present, but it won't be used
       tstes.queryTimestampDescendingSegmentTerminateEarlyYesGrouped(
-          cloudSolrClient, true /* appendKeyDescendingToSort */);
+          collectionName, cloudSolrClient, true /* appendKeyDescendingToSort */);
       // uses a sort order that is _not_ compatible with the merge sort order
       tstes.queryTimestampAscendingSegmentTerminateEarlyYes(
-          cloudSolrClient, true /* appendKeyDescendingToSort */);
+          collectionName, cloudSolrClient, true /* appendKeyDescendingToSort */);
     }
   }
 
@@ -154,7 +152,7 @@ public class TestSegmentSorting extends SolrCloudTestCase {
                 params(
                     "includeDynamic", "true",
                     "showDefaults", "true"))
-            .process(cloudSolrClient)
+            .process(cloudSolrClient, collectionName)
             .getField();
     assertEquals(true, schemaOpts.get("docValues"));
     assertEquals(false, schemaOpts.get("indexed"));
@@ -163,9 +161,9 @@ public class TestSegmentSorting extends SolrCloudTestCase {
     // add some documents
     final int numDocs = atLeast(1000);
     for (int id = 1; id <= numDocs; id++) {
-      cloudSolrClient.add(sdoc("id", id, updateField, random().nextInt(60)));
+      cloudSolrClient.add(collectionName, sdoc("id", id, updateField, random().nextInt(60)));
     }
-    cloudSolrClient.commit();
+    cloudSolrClient.commit(collectionName);
 
     // do some random iterations of replacing docs, atomic updates against segment sort field, and
     // commits (at this point we're just sanity checking no serious failures)
@@ -174,34 +172,40 @@ public class TestSegmentSorting extends SolrCloudTestCase {
       for (int i = 0; i < iterSize; i++) {
         // replace
         cloudSolrClient.add(
+            collectionName,
             sdoc("id", TestUtil.nextInt(random(), 1, numDocs), updateField, random().nextInt(60)));
         // atomic update
         cloudSolrClient.add(
+            collectionName,
             sdoc(
                 "id",
                 TestUtil.nextInt(random(), 1, numDocs),
                 updateField,
                 map("set", random().nextInt(60))));
       }
-      cloudSolrClient.commit();
+      cloudSolrClient.commit(collectionName);
     }
 
     // pick a random doc, and verify that doing an atomic update causes the docid to change
-    // ie: not an inplace update
+    // ie: not an in-place update
     final int id = TestUtil.nextInt(random(), 1, numDocs);
     final int oldDocId =
-        (Integer) cloudSolrClient.getById("" + id, params("fl", "[docid]")).get("[docid]");
+        (Integer)
+            cloudSolrClient
+                .getById(collectionName, "" + id, params("fl", "[docid]"))
+                .get("[docid]");
 
-    cloudSolrClient.add(sdoc("id", id, updateField, map("inc", "666")));
-    cloudSolrClient.commit();
+    cloudSolrClient.add(collectionName, sdoc("id", id, updateField, map("inc", "666")));
+    cloudSolrClient.commit(collectionName);
 
-    // loop incase we're waiting for a newSearcher to be opened
+    // loop in case we're waiting for a newSearcher to be opened
     int newDocId = -1;
     int attempts = 10;
     while ((newDocId < 0) && (0 < attempts--)) {
       SolrDocumentList docs =
           cloudSolrClient
               .query(
+                  collectionName,
                   params(
                       "q", "id:" + id,
                       "fl", "[docid]",
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestStressCloudBlindAtomicUpdates.java b/solr/core/src/test/org/apache/solr/cloud/TestStressCloudBlindAtomicUpdates.java
index 4efeed687d7..6eda1f7b15b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestStressCloudBlindAtomicUpdates.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestStressCloudBlindAtomicUpdates.java
@@ -74,8 +74,8 @@ public class TestStressCloudBlindAtomicUpdates extends SolrCloudTestCase {
   private static final String DEBUG_LABEL = MethodHandles.lookup().lookupClass().getName();
   private static final String COLLECTION_NAME = "test_col";
 
-  /** A basic client for operations at the cloud level, default collection will be set */
-  private static CloudSolrClient CLOUD_CLIENT;
+  /** A collection specific client for operations at the cloud level */
+  private static CloudSolrClient COLLECTION_CLIENT;
   /** One client per node */
   private static final ArrayList<SolrClient> CLIENTS = new ArrayList<>(5);
 
@@ -127,15 +127,14 @@ public class TestStressCloudBlindAtomicUpdates extends SolrCloudTestCase {
 
     configureCluster(numNodes).addConfig(configName, configDir).configure();
 
-    CLOUD_CLIENT = cluster.getSolrClient();
-    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    COLLECTION_CLIENT = cluster.getSolrClient(COLLECTION_NAME);
 
     CollectionAdminRequest.createCollection(COLLECTION_NAME, configName, numShards, repFactor)
         .withProperty("config", "solrconfig-tlog.xml")
         .withProperty("schema", "schema-minimal-atomic-stress.xml")
-        .process(CLOUD_CLIENT);
+        .process(COLLECTION_CLIENT);
 
-    waitForRecoveriesToFinish(CLOUD_CLIENT);
+    waitForRecoveriesToFinish(COLLECTION_CLIENT);
 
     CLIENTS.clear();
     for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
@@ -163,9 +162,9 @@ public class TestStressCloudBlindAtomicUpdates extends SolrCloudTestCase {
       ExecutorUtil.shutdownAndAwaitTermination(EXEC_SERVICE);
       EXEC_SERVICE = null;
     }
-    if (null != CLOUD_CLIENT) {
-      IOUtils.closeQuietly(CLOUD_CLIENT);
-      CLOUD_CLIENT = null;
+    if (null != COLLECTION_CLIENT) {
+      IOUtils.closeQuietly(COLLECTION_CLIENT);
+      COLLECTION_CLIENT = null;
     }
     for (SolrClient client : CLIENTS) {
       if (null == client) {
@@ -179,15 +178,15 @@ public class TestStressCloudBlindAtomicUpdates extends SolrCloudTestCase {
   @Before
   public void clearCloudCollection() throws Exception {
     TestInjection.reset();
-    waitForRecoveriesToFinish(CLOUD_CLIENT);
+    waitForRecoveriesToFinish(COLLECTION_CLIENT);
 
-    assertEquals(0, CLOUD_CLIENT.deleteByQuery("*:*").getStatus());
-    assertEquals(0, CLOUD_CLIENT.optimize().getStatus());
+    assertEquals(0, COLLECTION_CLIENT.deleteByQuery("*:*").getStatus());
+    assertEquals(0, COLLECTION_CLIENT.optimize().getStatus());
 
     assertEquals(
         "Collection should be empty!",
         0,
-        CLOUD_CLIENT.query(params("q", "*:*")).getResults().getNumFound());
+        COLLECTION_CLIENT.query(params("q", "*:*")).getResults().getNumFound());
 
     final int injectionPercentage = (int) Math.ceil((float) atLeast(1) / 2);
     testInjection = usually() ? "false:0" : ("true:" + injectionPercentage);
@@ -301,7 +300,7 @@ public class TestStressCloudBlindAtomicUpdates extends SolrCloudTestCase {
       // and we will inc/dec by random smaller ints, to ensure we never over/under flow
       final int initValue = random().nextInt();
       SolrInputDocument doc = doc(f("id", "" + id), f(numericFieldName, initValue));
-      UpdateResponse rsp = update(doc).process(CLOUD_CLIENT);
+      UpdateResponse rsp = update(doc).process(COLLECTION_CLIENT);
       assertEquals(doc + " => " + rsp, 0, rsp.getStatus());
       if (0 == id % DOC_ID_INCR) {
         expected[id / DOC_ID_INCR] = new AtomicLong(initValue);
@@ -310,9 +309,10 @@ public class TestStressCloudBlindAtomicUpdates extends SolrCloudTestCase {
     assertNotNull("Sanity Check no off-by-one in expected init: ", expected[expected.length - 1]);
 
     // sanity check index contents
-    waitForRecoveriesToFinish(CLOUD_CLIENT);
-    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
-    assertEquals(numDocsInIndex, CLOUD_CLIENT.query(params("q", "*:*")).getResults().getNumFound());
+    waitForRecoveriesToFinish(COLLECTION_CLIENT);
+    assertEquals(0, COLLECTION_CLIENT.commit().getStatus());
+    assertEquals(
+        numDocsInIndex, COLLECTION_CLIENT.query(params("q", "*:*")).getResults().getNumFound());
 
     startTestInjection();
 
@@ -354,7 +354,7 @@ public class TestStressCloudBlindAtomicUpdates extends SolrCloudTestCase {
         abortLatch.getCount());
 
     TestInjection.reset();
-    waitForRecoveriesToFinish(CLOUD_CLIENT);
+    waitForRecoveriesToFinish(COLLECTION_CLIENT);
 
     // check all the final index contents match our expectations
     int incorrectDocs = 0;
@@ -518,7 +518,7 @@ public class TestStressCloudBlindAtomicUpdates extends SolrCloudTestCase {
   public static SolrClient getRandClient(Random rand) {
     int numClients = CLIENTS.size();
     int idx = TestUtil.nextInt(rand, 0, numClients);
-    return (idx == numClients) ? CLOUD_CLIENT : CLIENTS.get(idx);
+    return (idx == numClients) ? COLLECTION_CLIENT : CLIENTS.get(idx);
   }
 
   public static void waitForRecoveriesToFinish(CloudSolrClient client) throws Exception {
@@ -532,12 +532,12 @@ public class TestStressCloudBlindAtomicUpdates extends SolrCloudTestCase {
    * Use the schema API to verify that the specified expected Field exists with those exact
    * attributes.
    *
-   * @see #CLOUD_CLIENT
+   * @see #COLLECTION_CLIENT
    */
   public static void checkExpectedSchemaField(Map<String, Object> expected) throws Exception {
     String fieldName = (String) expected.get("name");
     assertNotNull("expected contains no name: " + expected, fieldName);
-    FieldResponse rsp = new Field(fieldName).process(CLOUD_CLIENT);
+    FieldResponse rsp = new Field(fieldName).process(COLLECTION_CLIENT);
     assertNotNull("Field Null Response: " + fieldName, rsp);
     assertEquals("Field Status: " + fieldName + " => " + rsp, 0, rsp.getStatus());
     assertEquals("Field: " + fieldName, expected, rsp.getField());
@@ -547,13 +547,13 @@ public class TestStressCloudBlindAtomicUpdates extends SolrCloudTestCase {
    * Use the schema API to verify that the specified expected FieldType exists with those exact
    * attributes.
    *
-   * @see #CLOUD_CLIENT
+   * @see #COLLECTION_CLIENT
    */
   public static void checkExpectedSchemaType(Map<String, Object> expected) throws Exception {
 
     String typeName = (String) expected.get("name");
     assertNotNull("expected contains no type: " + expected, typeName);
-    FieldTypeResponse rsp = new FieldType(typeName).process(CLOUD_CLIENT);
+    FieldTypeResponse rsp = new FieldType(typeName).process(COLLECTION_CLIENT);
     assertNotNull("FieldType Null Response: " + typeName, rsp);
     assertEquals("FieldType Status: " + typeName + " => " + rsp, 0, rsp.getStatus());
     assertEquals("FieldType: " + typeName, expected, rsp.getFieldType().getAttributes());
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
index aa49e49663c..79f1f1b78cc 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
@@ -67,8 +67,8 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
 
   private static final String COLLECTION_NAME = "test_col";
 
-  /** A basic client for operations at the cloud level, default collection will be set */
-  private static CloudSolrClient CLOUD_CLIENT;
+  /** A collection specific client for operations at the cloud level */
+  private static CloudSolrClient COLLECTION_CLIENT;
 
   /** A client for talking directly to the leader of shard1 */
   private static SolrClient S_ONE_LEADER_CLIENT;
@@ -100,14 +100,13 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
 
     configureCluster(NUM_SERVERS).addConfig(configName, configDir.toPath()).configure();
 
-    CLOUD_CLIENT = cluster.getSolrClient();
-    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    COLLECTION_CLIENT = cluster.getSolrClient(COLLECTION_NAME);
 
     CollectionAdminRequest.createCollection(
             COLLECTION_NAME, configName, NUM_SHARDS, REPLICATION_FACTOR)
         .withProperty("config", "solrconfig-distrib-update-processor-chains.xml")
         .withProperty("schema", "schema15.xml") // string id for doc routing prefix
-        .process(CLOUD_CLIENT);
+        .process(COLLECTION_CLIENT);
 
     cluster.waitForActiveCollection(COLLECTION_NAME, NUM_SHARDS, REPLICATION_FACTOR * NUM_SHARDS);
 
@@ -170,17 +169,17 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
     // routing
     assertEquals(
         0,
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .add(doc(f("id", S_ONE_PRE + random().nextInt()), f("expected_shard_s", "shard1")))
             .getStatus());
     assertEquals(
         0,
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .add(doc(f("id", S_TWO_PRE + random().nextInt()), f("expected_shard_s", "shard2")))
             .getStatus());
-    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
+    assertEquals(0, COLLECTION_CLIENT.commit().getStatus());
     SolrDocumentList docs =
-        CLOUD_CLIENT
+        COLLECTION_CLIENT
             .query(
                 params(
                     "q", "*:*",
@@ -218,8 +217,8 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
     S_TWO_NON_LEADER_CLIENT = null;
     close(NO_COLLECTION_CLIENT);
     NO_COLLECTION_CLIENT = null;
-    close(CLOUD_CLIENT);
-    CLOUD_CLIENT = null;
+    close(COLLECTION_CLIENT);
+    COLLECTION_CLIENT = null;
   }
 
   private static void close(SolrClient client) throws IOException {
@@ -230,8 +229,8 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
 
   @Before
   public void clearCollection() throws Exception {
-    assertEquals(0, CLOUD_CLIENT.deleteByQuery("*:*").getStatus());
-    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
+    assertEquals(0, COLLECTION_CLIENT.deleteByQuery("*:*").getStatus());
+    assertEquals(0, COLLECTION_CLIENT.commit().getStatus());
   }
 
   public void testSanity() throws Exception {
@@ -239,15 +238,17 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
     // verify some basic sanity checking of indexing & querying across the collection
     // w/o using our custom update processor chain
 
-    assertEquals(0, CLOUD_CLIENT.add(doc(f("id", S_ONE_PRE + "1"), f("foo_i", 42))).getStatus());
-    assertEquals(0, CLOUD_CLIENT.add(doc(f("id", S_TWO_PRE + "2"), f("foo_i", 66))).getStatus());
-    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
+    assertEquals(
+        0, COLLECTION_CLIENT.add(doc(f("id", S_ONE_PRE + "1"), f("foo_i", 42))).getStatus());
+    assertEquals(
+        0, COLLECTION_CLIENT.add(doc(f("id", S_TWO_PRE + "2"), f("foo_i", 66))).getStatus());
+    assertEquals(0, COLLECTION_CLIENT.commit().getStatus());
 
     for (SolrClient c :
         Arrays.asList(
             S_ONE_LEADER_CLIENT, S_TWO_LEADER_CLIENT,
             S_ONE_NON_LEADER_CLIENT, S_TWO_NON_LEADER_CLIENT,
-            NO_COLLECTION_CLIENT, CLOUD_CLIENT)) {
+            NO_COLLECTION_CLIENT, COLLECTION_CLIENT)) {
       assertQueryDocIds(c, true, S_ONE_PRE + "1", S_TWO_PRE + "2");
       assertQueryDocIds(c, false, "id_not_exists");
 
@@ -298,7 +299,7 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
 
   //
   public void testVariousDeletesViaCloudClient() throws Exception {
-    testVariousDeletes(CLOUD_CLIENT);
+    testVariousDeletes(COLLECTION_CLIENT);
   }
 
   public void testVariousDeletesViaShard1LeaderClient() throws Exception {
@@ -463,7 +464,7 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
 
   //
   public void testVariousAddsViaCloudClient() throws Exception {
-    testVariousAdds(CLOUD_CLIENT);
+    testVariousAdds(COLLECTION_CLIENT);
   }
 
   public void testVariousAddsViaShard1LeaderClient() throws Exception {
@@ -984,7 +985,7 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
 
   //
   public void testAddsMixedWithDeletesViaCloudClient() throws Exception {
-    testAddsMixedWithDeletes(CLOUD_CLIENT);
+    testAddsMixedWithDeletes(COLLECTION_CLIENT);
   }
 
   public void testAddsMixedWithDeletesViaShard1LeaderClient() throws Exception {
@@ -1208,10 +1209,10 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
       assertEquals(
           client.toString() + " should " + (shouldExist ? "" : "not ") + "find id: " + id,
           (shouldExist ? 1 : 0),
-          CLOUD_CLIENT.query(params("q", "{!term f=id}" + id)).getResults().getNumFound());
+          COLLECTION_CLIENT.query(params("q", "{!term f=id}" + id)).getResults().getNumFound());
     }
-    if (!CLOUD_CLIENT.equals(client)) {
-      assertQueryDocIds(CLOUD_CLIENT, shouldExist, ids);
+    if (!COLLECTION_CLIENT.equals(client)) {
+      assertQueryDocIds(COLLECTION_CLIENT, shouldExist, ids);
     }
   }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java
index 30a81241c2f..915e63e9fe4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorRandomCloud.java
@@ -70,8 +70,8 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
 
   private static final String COLLECTION_NAME = "test_col";
 
-  /** A basic client for operations at the cloud level, default collection will be set */
-  private static CloudSolrClient CLOUD_CLIENT;
+  /** A collection specific client for operations at the cloud level */
+  private static CloudSolrClient COLLECTION_CLIENT;
   /** one SolrClient for each server */
   private static List<SolrClient> NODE_CLIENTS;
 
@@ -98,12 +98,11 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
     collectionProperties.put("config", "solrconfig-distrib-update-processor-chains.xml");
     collectionProperties.put("schema", "schema15.xml"); // string id
 
-    CLOUD_CLIENT = cluster.getSolrClient();
-    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    COLLECTION_CLIENT = cluster.getSolrClient(COLLECTION_NAME);
 
     CollectionAdminRequest.createCollection(COLLECTION_NAME, configName, numShards, repFactor)
         .setProperties(collectionProperties)
-        .process(CLOUD_CLIENT);
+        .process(COLLECTION_CLIENT);
 
     cluster.waitForActiveCollection(COLLECTION_NAME, numShards, numShards * repFactor);
 
@@ -124,8 +123,12 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
   @Before
   public void deleteAllDocs() throws Exception {
     assertEquals(
-        0, update(params("commit", "true")).deleteByQuery("*:*").process(CLOUD_CLIENT).getStatus());
-    assertEquals("index should be empty", 0L, countDocs(CLOUD_CLIENT));
+        0,
+        update(params("commit", "true"))
+            .deleteByQuery("*:*")
+            .process(COLLECTION_CLIENT)
+            .getStatus());
+    assertEquals("index should be empty", 0L, countDocs(COLLECTION_CLIENT));
   }
 
   @AfterClass
@@ -136,10 +139,10 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
       }
     }
     NODE_CLIENTS = null;
-    if (CLOUD_CLIENT != null) {
-      CLOUD_CLIENT.close();
+    if (COLLECTION_CLIENT != null) {
+      COLLECTION_CLIENT.close();
     }
-    CLOUD_CLIENT = null;
+    COLLECTION_CLIENT = null;
   }
 
   public void testRandomUpdates() throws Exception {
@@ -266,7 +269,7 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
 
       final SolrClient client =
           random().nextBoolean()
-              ? CLOUD_CLIENT
+              ? COLLECTION_CLIENT
               : NODE_CLIENTS.get(TestUtil.nextInt(random(), 0, NODE_CLIENTS.size() - 1));
 
       final UpdateResponse rsp = req.process(client);
@@ -279,10 +282,10 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
         log.info("END ITER #{}, expecting #docs: {}", i, expectedDocIds.cardinality());
       }
 
-      assertEquals("post update commit failed?", 0, CLOUD_CLIENT.commit().getStatus());
+      assertEquals("post update commit failed?", 0, COLLECTION_CLIENT.commit().getStatus());
 
       for (int j = 0; j < 5; j++) {
-        if (expectedDocIds.cardinality() == countDocs(CLOUD_CLIENT)) {
+        if (expectedDocIds.cardinality() == countDocs(COLLECTION_CLIENT)) {
           break;
         }
         log.info("sleeping to give searchers a chance to re-open #{}", j);
@@ -290,7 +293,7 @@ public class TestTolerantUpdateProcessorRandomCloud extends SolrCloudTestCase {
       }
 
       // check the index contents against our expectations
-      final BitSet actualDocIds = allDocs(CLOUD_CLIENT, maxDocId);
+      final BitSet actualDocIds = allDocs(COLLECTION_CLIENT, maxDocId);
       if (expectedDocIds.cardinality() != actualDocIds.cardinality()) {
         log.error(
             "cardinality mismatch: expected {} BUT actual {}",
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/ShardSplitTest.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/ShardSplitTest.java
index e3317c44120..da5ce1b7cbb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/ShardSplitTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/ShardSplitTest.java
@@ -21,10 +21,12 @@ import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
@@ -148,10 +150,16 @@ public class ShardSplitTest extends BasicDistributedZkTest {
         .waitForState(
             collectionName, 30, TimeUnit.SECONDS, SolrCloudTestCase.activeClusterShape(1, 1));
 
+    var builder =
+        new RandomizingCloudSolrClientBuilder(
+            Collections.singletonList(zkServer.getZkAddress()), Optional.empty());
+
     try (CloudSolrClient client =
-        getCloudSolrClient(
-            zkServer.getZkAddress(), true, ((CloudLegacySolrClient) cloudClient).getHttpClient())) {
-      client.setDefaultCollection(collectionName);
+        builder
+            .withDefaultCollection(collectionName)
+            .sendUpdatesOnlyToShardLeaders()
+            .withHttpClient(((CloudLegacySolrClient) cloudClient).getHttpClient())
+            .build()) {
       StoppableIndexingThread thread =
           new StoppableIndexingThread(controlClient, client, "i1", true);
       try {
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/SplitByPrefixTest.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/SplitByPrefixTest.java
index 3991c1d229b..a09b350ecd1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/SplitByPrefixTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/SplitByPrefixTest.java
@@ -150,8 +150,7 @@ public class SplitByPrefixTest extends SolrCloudTestCase {
 
     cluster.waitForActiveCollection(COLLECTION_NAME, 1, 1);
 
-    CloudSolrClient client = cluster.getSolrClient();
-    client.setDefaultCollection(COLLECTION_NAME);
+    CloudSolrClient client = cluster.getSolrClient(COLLECTION_NAME);
 
     // splitting an empty collection by prefix should still work (i.e. fall back to old method of
     // just dividing the hash range
diff --git a/solr/core/src/test/org/apache/solr/handler/component/TestTrackingShardHandlerFactory.java b/solr/core/src/test/org/apache/solr/handler/component/TestTrackingShardHandlerFactory.java
index 6af89ef4f7d..4c9626618ec 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/TestTrackingShardHandlerFactory.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/TestTrackingShardHandlerFactory.java
@@ -22,7 +22,6 @@ import java.util.Map;
 import org.apache.solr.BaseDistributedSearchTestCase;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrQuery;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -78,9 +77,6 @@ public class TestTrackingShardHandlerFactory extends AbstractFullDistribZkTestBa
     assertEquals(
         "Unexpected number of core admin requests were found", 2, coreAdminRequests.size());
 
-    CloudSolrClient client = cloudClient;
-
-    client.setDefaultCollection(collectionName);
     /*
     hash of b is 95de7e03 high bits=2 shard=shard1
     hash of e is 656c4367 high bits=1 shard=shard2
@@ -90,28 +86,40 @@ public class TestTrackingShardHandlerFactory extends AbstractFullDistribZkTestBa
       doc.addField("id", (i % 2 == 0 ? "b!" : "e!") + i);
       doc.addField("a_i", i);
       doc.addField("a_t", "text_" + i);
-      client.add(doc);
+      cloudClient.add(collectionName, doc);
     }
-    client.commit();
+    cloudClient.commit(collectionName);
 
-    client.query(new SolrQuery("*:*"));
+    cloudClient.query(collectionName, new SolrQuery("*:*"));
 
     TrackingShardHandlerFactory.ShardRequestAndParams getTopIdsRequest =
         trackingQueue.getShardRequestByPurpose(
-            ZkStateReader.from(client), collectionName, "shard1", ShardRequest.PURPOSE_GET_TOP_IDS);
+            ZkStateReader.from(cloudClient),
+            collectionName,
+            "shard1",
+            ShardRequest.PURPOSE_GET_TOP_IDS);
     assertNotNull(getTopIdsRequest);
     getTopIdsRequest =
         trackingQueue.getShardRequestByPurpose(
-            ZkStateReader.from(client), collectionName, "shard2", ShardRequest.PURPOSE_GET_TOP_IDS);
+            ZkStateReader.from(cloudClient),
+            collectionName,
+            "shard2",
+            ShardRequest.PURPOSE_GET_TOP_IDS);
     assertNotNull(getTopIdsRequest);
 
     TrackingShardHandlerFactory.ShardRequestAndParams getFieldsRequest =
         trackingQueue.getShardRequestByPurpose(
-            ZkStateReader.from(client), collectionName, "shard1", ShardRequest.PURPOSE_GET_FIELDS);
+            ZkStateReader.from(cloudClient),
+            collectionName,
+            "shard1",
+            ShardRequest.PURPOSE_GET_FIELDS);
     assertNotNull(getFieldsRequest);
     getFieldsRequest =
         trackingQueue.getShardRequestByPurpose(
-            ZkStateReader.from(client), collectionName, "shard2", ShardRequest.PURPOSE_GET_FIELDS);
+            ZkStateReader.from(cloudClient),
+            collectionName,
+            "shard2",
+            ShardRequest.PURPOSE_GET_FIELDS);
     assertNotNull(getFieldsRequest);
 
     int numRequests = 0;
@@ -138,7 +146,7 @@ public class TestTrackingShardHandlerFactory extends AbstractFullDistribZkTestBa
     }
 
     // make another request and verify
-    client.query(new SolrQuery("*:*"));
+    cloudClient.query(collectionName, new SolrQuery("*:*"));
     numRequests = 0;
     allRequests = trackingQueue.getAllRequests();
     for (Map.Entry<String, List<TrackingShardHandlerFactory.ShardRequestAndParams>> entry :
diff --git a/solr/core/src/test/org/apache/solr/response/transform/TestSubQueryTransformerDistrib.java b/solr/core/src/test/org/apache/solr/response/transform/TestSubQueryTransformerDistrib.java
index 40bc38fd1a3..899d6ebfa88 100644
--- a/solr/core/src/test/org/apache/solr/response/transform/TestSubQueryTransformerDistrib.java
+++ b/solr/core/src/test/org/apache/solr/response/transform/TestSubQueryTransformerDistrib.java
@@ -82,7 +82,6 @@ public class TestSubQueryTransformerDistrib extends SolrCloudTestCase {
         .process(cluster.getSolrClient());
 
     CloudSolrClient client = cluster.getSolrClient();
-    client.setDefaultCollection(people);
 
     ZkStateReader zkStateReader = ZkStateReader.from(client);
     AbstractDistribZkTestBase.waitForRecoveriesToFinish(people, zkStateReader, true, true, 30);
diff --git a/solr/core/src/test/org/apache/solr/search/CurrencyRangeFacetCloudTest.java b/solr/core/src/test/org/apache/solr/search/CurrencyRangeFacetCloudTest.java
index 0d8ac7a7c88..2d22f90db5a 100644
--- a/solr/core/src/test/org/apache/solr/search/CurrencyRangeFacetCloudTest.java
+++ b/solr/core/src/test/org/apache/solr/search/CurrencyRangeFacetCloudTest.java
@@ -75,8 +75,6 @@ public class CurrencyRangeFacetCloudTest extends SolrCloudTestCase {
                 .process(cluster.getSolrClient()))
             .getStatus());
 
-    cluster.getSolrClient().setDefaultCollection(COLLECTION);
-
     // we're indexing each Currency value in 3 docs, each with a diff 'x_s' field value
     // use modulo to pick the values, so we don't add the docs in strict order of either VALUES of
     // STR_VALS (that way if we want ot filter by id later, it's an independent variable)
@@ -86,10 +84,10 @@ public class CurrencyRangeFacetCloudTest extends SolrCloudTestCase {
       assertEquals(
           0,
           (new UpdateRequest().add(sdoc("id", "" + id, "x_s", x, FIELD, val)))
-              .process(cluster.getSolrClient())
+              .process(cluster.getSolrClient(COLLECTION))
               .getStatus());
     }
-    assertEquals(0, cluster.getSolrClient().commit().getStatus());
+    assertEquals(0, cluster.getSolrClient(COLLECTION).commit().getStatus());
   }
 
   public void testSimpleRangeFacetsOfSymmetricRates() throws Exception {
@@ -137,7 +135,7 @@ public class CurrencyRangeFacetCloudTest extends SolrCloudTestCase {
                 args.get(2),
                 "f." + FIELD + ".facet.range.other",
                 "all");
-        QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+        QueryResponse rsp = cluster.getSolrClient(COLLECTION).query(solrQuery);
         try {
           assertEquals(NUM_DOCS, rsp.getResults().getNumFound());
 
@@ -199,7 +197,7 @@ public class CurrencyRangeFacetCloudTest extends SolrCloudTestCase {
                     + "', end:'"
                     + args.get(2)
                     + "', other:all}}");
-        rsp = cluster.getSolrClient().query(solrQuery);
+        rsp = cluster.getSolrClient(COLLECTION).query(solrQuery);
         try {
           assertEquals(NUM_DOCS, rsp.getResults().getNumFound());
 
@@ -265,7 +263,7 @@ public class CurrencyRangeFacetCloudTest extends SolrCloudTestCase {
               "22,EUR",
               "f." + FIELD + ".facet.range.other",
               "all");
-      final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+      final QueryResponse rsp = cluster.getSolrClient(COLLECTION).query(solrQuery);
       try {
         assertEquals(NUM_DOCS, rsp.getResults().getNumFound());
         @SuppressWarnings({"rawtypes"})
@@ -324,7 +322,7 @@ public class CurrencyRangeFacetCloudTest extends SolrCloudTestCase {
                   + (use_mincount ? 3 : 0)
                   + ", "
                   + "        gap:'2,EUR', end:'22,EUR', other:all}}");
-      final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+      final QueryResponse rsp = cluster.getSolrClient(COLLECTION).query(solrQuery);
       try {
         assertEquals(NUM_DOCS, rsp.getResults().getNumFound());
 
@@ -387,7 +385,7 @@ public class CurrencyRangeFacetCloudTest extends SolrCloudTestCase {
         expectThrows(
             SolrException.class,
             () -> {
-              final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+              final QueryResponse rsp = cluster.getSolrClient(COLLECTION).query(solrQuery);
             });
     assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, ex.code());
     assertTrue(ex.getMessage(), ex.getMessage().contains(expected));
@@ -411,7 +409,7 @@ public class CurrencyRangeFacetCloudTest extends SolrCloudTestCase {
         expectThrows(
             SolrException.class,
             () -> {
-              final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+              final QueryResponse rsp = cluster.getSolrClient(COLLECTION).query(solrQuery);
             });
     assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, ex.code());
     assertTrue(ex.getMessage(), ex.getMessage().contains(expected));
@@ -455,7 +453,7 @@ public class CurrencyRangeFacetCloudTest extends SolrCloudTestCase {
                   + "        facet: { foo:{ type:terms, field:x_s, "
                   + "                       refine:true, limit:2, overrequest:0"
                   + " } } } }");
-      final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+      final QueryResponse rsp = cluster.getSolrClient(COLLECTION).query(solrQuery);
       try {
         // this top level result count sanity check that should vary based on how we are filtering
         // our facets...
@@ -566,7 +564,7 @@ public class CurrencyRangeFacetCloudTest extends SolrCloudTestCase {
                   + FIELD
                   + ", other:all, "
                   + "                       start:'8,EUR', gap:'2,EUR', end:'22,EUR' }} } }");
-      final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+      final QueryResponse rsp = cluster.getSolrClient(COLLECTION).query(solrQuery);
       try {
         // this top level result count sanity check that should vary based on how we are filtering
         // our facets...
diff --git a/solr/core/src/test/org/apache/solr/search/TestTaskManagement.java b/solr/core/src/test/org/apache/solr/search/TestTaskManagement.java
index 097906a3067..187fe048e12 100644
--- a/solr/core/src/test/org/apache/solr/search/TestTaskManagement.java
+++ b/solr/core/src/test/org/apache/solr/search/TestTaskManagement.java
@@ -72,7 +72,6 @@ public class TestTaskManagement extends SolrCloudTestCase {
         .setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
         .process(cluster.getSolrClient());
     cluster.waitForActiveCollection(COLLECTION_NAME, 2, 2);
-    cluster.getSolrClient().setDefaultCollection(COLLECTION_NAME);
 
     queryExecutor = ExecutorUtil.newMDCAwareCachedThreadPool("TestTaskManagement-Query");
     cancelExecutor = ExecutorUtil.newMDCAwareCachedThreadPool("TestTaskManagement-Cancel");
@@ -88,8 +87,8 @@ public class TestTaskManagement extends SolrCloudTestCase {
       docs.add(doc);
     }
 
-    cluster.getSolrClient().add(docs);
-    cluster.getSolrClient().commit();
+    cluster.getSolrClient(COLLECTION_NAME).add(docs);
+    cluster.getSolrClient(COLLECTION_NAME).commit();
   }
 
   @After
@@ -111,7 +110,7 @@ public class TestTaskManagement extends SolrCloudTestCase {
 
     GenericSolrRequest request =
         new GenericSolrRequest(SolrRequest.METHOD.GET, "/tasks/cancel", params);
-    NamedList<Object> queryResponse = cluster.getSolrClient().request(request);
+    NamedList<Object> queryResponse = cluster.getSolrClient(COLLECTION_NAME).request(request);
 
     assertEquals("Query with queryID foobar not found", queryResponse.get("status"));
     assertEquals(404, queryResponse.get("responseCode"));
@@ -185,7 +184,7 @@ public class TestTaskManagement extends SolrCloudTestCase {
   private NamedList<String> listTasks() throws SolrServerException, IOException {
     NamedList<Object> response =
         cluster
-            .getSolrClient()
+            .getSolrClient(COLLECTION_NAME)
             .request(new GenericSolrRequest(SolrRequest.METHOD.GET, "/tasks/list", null));
     return (NamedList<String>) response.get("taskList");
   }
@@ -197,7 +196,7 @@ public class TestTaskManagement extends SolrCloudTestCase {
 
     GenericSolrRequest request =
         new GenericSolrRequest(SolrRequest.METHOD.GET, "/tasks/list", params);
-    NamedList<Object> queryResponse = cluster.getSolrClient().request(request);
+    NamedList<Object> queryResponse = cluster.getSolrClient(COLLECTION_NAME).request(request);
 
     String result = (String) queryResponse.get("taskStatus");
 
@@ -217,7 +216,7 @@ public class TestTaskManagement extends SolrCloudTestCase {
           try {
             NamedList<Object> queryResponse;
 
-            queryResponse = cluster.getSolrClient().request(request);
+            queryResponse = cluster.getSolrClient(COLLECTION_NAME).request(request);
 
             int responseCode = (int) queryResponse.get("responseCode");
 
@@ -245,7 +244,7 @@ public class TestTaskManagement extends SolrCloudTestCase {
 
     SolrRequest<?> request = new QueryRequest(params);
 
-    cluster.getSolrClient().request(request);
+    cluster.getSolrClient(COLLECTION_NAME).request(request);
   }
 
   public CompletableFuture<Void> executeQueryAsync(String queryId) {
diff --git a/solr/core/src/test/org/apache/solr/search/facet/RangeFacetCloudTest.java b/solr/core/src/test/org/apache/solr/search/facet/RangeFacetCloudTest.java
index 4de05a620c7..20c0b34f10c 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/RangeFacetCloudTest.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/RangeFacetCloudTest.java
@@ -30,6 +30,7 @@ import java.util.Map.Entry;
 import java.util.stream.Collectors;
 import org.apache.lucene.tests.util.TestUtil;
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
@@ -37,7 +38,9 @@ import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.FacetParams.FacetRangeOther;
+import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
 /**
@@ -78,6 +81,9 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
   private static final Map<String, Integer>[] TERM_MODEL =
       (Map<String, Integer>[]) Array.newInstance(Map.class, NUM_RANGE_VALUES);
 
+  /** A basic client for operations at the cloud level, default collection will be set */
+  private static CloudSolrClient CLOUD_CLIENT;
+
   @BeforeClass
   public static void setupCluster() throws Exception {
     final int numShards = TestUtil.nextInt(random(), 1, 5);
@@ -96,7 +102,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                 .process(cluster.getSolrClient()))
             .getStatus());
 
-    cluster.getSolrClient().setDefaultCollection(COLLECTION);
+    CLOUD_CLIENT = cluster.basicSolrClientBuilder().withDefaultCollection(COLLECTION).build();
 
     final int numDocs = atLeast(1000);
     final int maxTermId = atLeast(TERM_VALUES_RANDOMIZER);
@@ -117,9 +123,9 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
       RANGE_MODEL[rangeVal]++;
       TERM_MODEL[rangeVal].merge(termVal, 1, Integer::sum);
 
-      assertEquals(0, (new UpdateRequest().add(doc)).process(cluster.getSolrClient()).getStatus());
+      assertEquals(0, (new UpdateRequest().add(doc)).process(CLOUD_CLIENT).getStatus());
     }
-    assertEquals(0, cluster.getSolrClient().commit().getStatus());
+    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
   }
 
   public void testInclude_Lower() throws Exception {
@@ -145,7 +151,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                       + subFacet
                       + " } }");
 
-          final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+          final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
           try {
             @SuppressWarnings({"unchecked"})
             final NamedList<Object> foo =
@@ -197,7 +203,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                       + subFacet
                       + " } }");
 
-          final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+          final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
           try {
             @SuppressWarnings({"unchecked"})
             final NamedList<Object> foo =
@@ -243,7 +249,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                       + subFacet
                       + " } }");
 
-          final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+          final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
           try {
             @SuppressWarnings({"unchecked"})
             final NamedList<Object> foo =
@@ -277,7 +283,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
             "true",
             "json.facet",
             "{unique_foo:\"unique(" + STR_FIELD + ")\"}");
-    final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+    final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
     // response shouldn't contain header as omitHeader is set to true
     assertNull(rsp.getResponseHeader());
   }
@@ -303,7 +309,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                     + subFacet
                     + " } }");
 
-        final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+        final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
         try {
           @SuppressWarnings({"unchecked"})
           final NamedList<Object> foo =
@@ -346,7 +352,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                     + subFacet
                     + " } }");
 
-        final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+        final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
         try {
           @SuppressWarnings({"unchecked"})
           final NamedList<Object> foo =
@@ -390,7 +396,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                     + subFacet
                     + " } }");
 
-        final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+        final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
         try {
           @SuppressWarnings({"unchecked"})
           final NamedList<Object> foo =
@@ -442,7 +448,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                       + subFacet
                       + " } }");
 
-          final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+          final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
           try {
             @SuppressWarnings({"unchecked"})
             final NamedList<Object> foo =
@@ -492,7 +498,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                       + subFacet
                       + " } }");
 
-          final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+          final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
           try {
             @SuppressWarnings({"unchecked"})
             final NamedList<Object> foo =
@@ -542,7 +548,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                       + subFacet
                       + " } }");
 
-          final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+          final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
           try {
             @SuppressWarnings({"unchecked"})
             final NamedList<Object> foo =
@@ -595,7 +601,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                       + subFacet
                       + " } }");
 
-          final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+          final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
           try {
             @SuppressWarnings({"unchecked"})
             final NamedList<Object> foo =
@@ -659,7 +665,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                       + subFacet
                       + " } }");
 
-          QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+          QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
           try {
             @SuppressWarnings({"unchecked"})
             final NamedList<Object> foo =
@@ -711,7 +717,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                       + subFacet
                       + " } }");
 
-          rsp = cluster.getSolrClient().query(solrQuery);
+          rsp = CLOUD_CLIENT.query(solrQuery);
           try {
             @SuppressWarnings({"unchecked"})
             final NamedList<Object> foo =
@@ -772,7 +778,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                       + subFacet
                       + " } }");
 
-          final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+          final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
           try {
             @SuppressWarnings({"unchecked"})
             final NamedList<Object> foo =
@@ -825,7 +831,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                     + subFacet
                     + " } }");
 
-        final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+        final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
         try {
           @SuppressWarnings({"unchecked"})
           final NamedList<Object> foo =
@@ -875,7 +881,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                     + subFacet
                     + " } }");
 
-        final QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+        final QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
         try {
           @SuppressWarnings({"unchecked"})
           final NamedList<Object> foo =
@@ -919,7 +925,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                   + subFacet
                   + " } }");
 
-      QueryResponse rsp = cluster.getSolrClient().query(solrQuery);
+      QueryResponse rsp = CLOUD_CLIENT.query(solrQuery);
       try {
         @SuppressWarnings({"unchecked"})
         final NamedList<Object> foo =
@@ -966,7 +972,7 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
                   + subFacet
                   + " } }");
 
-      rsp = cluster.getSolrClient().query(solrQuery);
+      rsp = CLOUD_CLIENT.query(solrQuery);
       try {
         @SuppressWarnings({"unchecked"})
         final NamedList<Object> foo =
@@ -1211,4 +1217,9 @@ public class RangeFacetCloudTest extends SolrCloudTestCase {
     }
     return ", other:" + val;
   }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    IOUtils.closeQuietly(CLOUD_CLIENT);
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetJoinDomain.java b/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetJoinDomain.java
index d5227ba28f7..1da1507a8de 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetJoinDomain.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetJoinDomain.java
@@ -43,6 +43,7 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.embedded.JettySolrRunner;
 import org.junit.AfterClass;
@@ -117,8 +118,7 @@ public class TestCloudJSONFacetJoinDomain extends SolrCloudTestCase {
         .setProperties(collectionProperties)
         .process(cluster.getSolrClient());
 
-    CLOUD_CLIENT = cluster.getSolrClient();
-    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    CLOUD_CLIENT = cluster.basicSolrClientBuilder().withDefaultCollection(COLLECTION_NAME).build();
 
     waitForRecoveriesToFinish(CLOUD_CLIENT);
 
@@ -189,12 +189,10 @@ public class TestCloudJSONFacetJoinDomain extends SolrCloudTestCase {
 
   @AfterClass
   public static void afterClass() throws Exception {
-    if (null != CLOUD_CLIENT) {
-      CLOUD_CLIENT.close();
-      CLOUD_CLIENT = null;
-    }
+    IOUtils.closeQuietly(CLOUD_CLIENT);
+
     for (SolrClient client : CLIENTS) {
-      client.close();
+      IOUtils.closeQuietly(client);
     }
     CLIENTS.clear();
   }
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetSKG.java b/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetSKG.java
index bbc58843195..0748978e659 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetSKG.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetSKG.java
@@ -43,6 +43,7 @@ import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.embedded.JettySolrRunner;
 import org.junit.AfterClass;
@@ -143,8 +144,7 @@ public class TestCloudJSONFacetSKG extends SolrCloudTestCase {
         .setProperties(collectionProperties)
         .process(cluster.getSolrClient());
 
-    CLOUD_CLIENT = cluster.getSolrClient();
-    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    CLOUD_CLIENT = cluster.basicSolrClientBuilder().withDefaultCollection(COLLECTION_NAME).build();
 
     waitForRecoveriesToFinish(CLOUD_CLIENT);
 
@@ -241,12 +241,10 @@ public class TestCloudJSONFacetSKG extends SolrCloudTestCase {
 
   @AfterClass
   public static void afterClass() throws Exception {
-    if (null != CLOUD_CLIENT) {
-      CLOUD_CLIENT.close();
-      CLOUD_CLIENT = null;
-    }
+    IOUtils.closeQuietly(CLOUD_CLIENT);
+
     for (SolrClient client : CLIENTS) {
-      client.close();
+      IOUtils.closeQuietly(client);
     }
     CLIENTS.clear();
   }
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetSKGEquiv.java b/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetSKGEquiv.java
index eee58309ef3..213c1ea6838 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetSKGEquiv.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestCloudJSONFacetSKGEquiv.java
@@ -46,6 +46,7 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.embedded.JettySolrRunner;
 import org.junit.AfterClass;
@@ -134,8 +135,7 @@ public class TestCloudJSONFacetSKGEquiv extends SolrCloudTestCase {
         .setProperties(collectionProperties)
         .process(cluster.getSolrClient());
 
-    CLOUD_CLIENT = cluster.getSolrClient();
-    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    CLOUD_CLIENT = cluster.basicSolrClientBuilder().withDefaultCollection(COLLECTION_NAME).build();
 
     waitForRecoveriesToFinish(CLOUD_CLIENT);
 
@@ -231,12 +231,9 @@ public class TestCloudJSONFacetSKGEquiv extends SolrCloudTestCase {
 
   @AfterClass
   public static void afterClass() throws Exception {
-    if (null != CLOUD_CLIENT) {
-      CLOUD_CLIENT.close();
-      CLOUD_CLIENT = null;
-    }
+    IOUtils.closeQuietly(CLOUD_CLIENT);
     for (SolrClient client : CLIENTS) {
-      client.close();
+      IOUtils.closeQuietly(client);
     }
     CLIENTS.clear();
   }
diff --git a/solr/core/src/test/org/apache/solr/search/join/TestCloudNestedDocsSort.java b/solr/core/src/test/org/apache/solr/search/join/TestCloudNestedDocsSort.java
index 5f5a03362c4..bd2501b0a11 100644
--- a/solr/core/src/test/org/apache/solr/search/join/TestCloudNestedDocsSort.java
+++ b/solr/core/src/test/org/apache/solr/search/join/TestCloudNestedDocsSort.java
@@ -34,6 +34,7 @@ import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.IOUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -66,8 +67,7 @@ public class TestCloudNestedDocsSort extends SolrCloudTestCase {
         .withProperty("schema", "schema.xml")
         .process(cluster.getSolrClient());
 
-    client = cluster.getSolrClient();
-    client.setDefaultCollection("collection1");
+    client = cluster.basicSolrClientBuilder().withDefaultCollection("collection1").build();
 
     ZkStateReader zkStateReader = ZkStateReader.from(client);
     AbstractDistribZkTestBase.waitForRecoveriesToFinish(
@@ -131,7 +131,7 @@ public class TestCloudNestedDocsSort extends SolrCloudTestCase {
 
   @AfterClass
   public static void cleanUpAfterClass() {
-    client = null;
+    IOUtils.closeQuietly(client);
   }
 
   @Test
diff --git a/solr/core/src/test/org/apache/solr/servlet/TestRequestRateLimiter.java b/solr/core/src/test/org/apache/solr/servlet/TestRequestRateLimiter.java
index b3740e8849c..c9b62b2b3c5 100644
--- a/solr/core/src/test/org/apache/solr/servlet/TestRequestRateLimiter.java
+++ b/solr/core/src/test/org/apache/solr/servlet/TestRequestRateLimiter.java
@@ -26,6 +26,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -51,105 +52,106 @@ public class TestRequestRateLimiter extends SolrCloudTestCase {
 
   @Test
   public void testConcurrentQueries() throws Exception {
-    CloudSolrClient client = cluster.getSolrClient();
-    client.setDefaultCollection(FIRST_COLLECTION);
-
-    CollectionAdminRequest.createCollection(FIRST_COLLECTION, 1, 1).process(client);
-    cluster.waitForActiveCollection(FIRST_COLLECTION, 1, 1);
-
-    SolrDispatchFilter solrDispatchFilter = cluster.getJettySolrRunner(0).getSolrDispatchFilter();
-
-    RateLimiterConfig rateLimiterConfig =
-        new RateLimiterConfig(
-            SolrRequest.SolrRequestType.QUERY,
-            true,
-            1,
-            DEFAULT_SLOT_ACQUISITION_TIMEOUT_MS,
-            5 /* allowedRequests */,
-            true /* isSlotBorrowing */);
-    // We are fine with a null FilterConfig here since we ensure that MockBuilder never invokes its
-    // parent here
-    RateLimitManager.Builder builder =
-        new MockBuilder(
-            null /* dummy SolrZkClient */, new MockRequestRateLimiter(rateLimiterConfig));
-    RateLimitManager rateLimitManager = builder.build();
-
-    solrDispatchFilter.replaceRateLimitManager(rateLimitManager);
-
-    int numDocs = TEST_NIGHTLY ? 10000 : 100;
-
-    processTest(client, numDocs, 350 /* number of queries */);
-
-    MockRequestRateLimiter mockQueryRateLimiter =
-        (MockRequestRateLimiter)
-            rateLimitManager.getRequestRateLimiter(SolrRequest.SolrRequestType.QUERY);
-
-    assertEquals(350, mockQueryRateLimiter.incomingRequestCount.get());
-
-    assertTrue(mockQueryRateLimiter.acceptedNewRequestCount.get() > 0);
-    assertTrue(
-        (mockQueryRateLimiter.acceptedNewRequestCount.get()
-                == mockQueryRateLimiter.incomingRequestCount.get()
-            || mockQueryRateLimiter.rejectedRequestCount.get() > 0));
-    assertEquals(
-        mockQueryRateLimiter.incomingRequestCount.get(),
-        mockQueryRateLimiter.acceptedNewRequestCount.get()
-            + mockQueryRateLimiter.rejectedRequestCount.get());
+    try (CloudSolrClient client =
+        cluster.basicSolrClientBuilder().withDefaultCollection(FIRST_COLLECTION).build()) {
+
+      CollectionAdminRequest.createCollection(FIRST_COLLECTION, 1, 1).process(client);
+      cluster.waitForActiveCollection(FIRST_COLLECTION, 1, 1);
+
+      SolrDispatchFilter solrDispatchFilter = cluster.getJettySolrRunner(0).getSolrDispatchFilter();
+
+      RateLimiterConfig rateLimiterConfig =
+          new RateLimiterConfig(
+              SolrRequest.SolrRequestType.QUERY,
+              true,
+              1,
+              DEFAULT_SLOT_ACQUISITION_TIMEOUT_MS,
+              5 /* allowedRequests */,
+              true /* isSlotBorrowing */);
+      // We are fine with a null FilterConfig here since we ensure that MockBuilder never invokes
+      // its parent here
+      RateLimitManager.Builder builder =
+          new MockBuilder(
+              null /* dummy SolrZkClient */, new MockRequestRateLimiter(rateLimiterConfig));
+      RateLimitManager rateLimitManager = builder.build();
+
+      solrDispatchFilter.replaceRateLimitManager(rateLimitManager);
+
+      int numDocs = TEST_NIGHTLY ? 10000 : 100;
+
+      processTest(client, numDocs, 350 /* number of queries */);
+
+      MockRequestRateLimiter mockQueryRateLimiter =
+          (MockRequestRateLimiter)
+              rateLimitManager.getRequestRateLimiter(SolrRequest.SolrRequestType.QUERY);
+
+      assertEquals(350, mockQueryRateLimiter.incomingRequestCount.get());
+
+      assertTrue(mockQueryRateLimiter.acceptedNewRequestCount.get() > 0);
+      assertTrue(
+          (mockQueryRateLimiter.acceptedNewRequestCount.get()
+                  == mockQueryRateLimiter.incomingRequestCount.get()
+              || mockQueryRateLimiter.rejectedRequestCount.get() > 0));
+      assertEquals(
+          mockQueryRateLimiter.incomingRequestCount.get(),
+          mockQueryRateLimiter.acceptedNewRequestCount.get()
+              + mockQueryRateLimiter.rejectedRequestCount.get());
+    }
   }
 
   @Nightly
   public void testSlotBorrowing() throws Exception {
-    CloudSolrClient client = cluster.getSolrClient();
-    client.setDefaultCollection(SECOND_COLLECTION);
-
-    CollectionAdminRequest.createCollection(SECOND_COLLECTION, 1, 1).process(client);
-    cluster.waitForActiveCollection(SECOND_COLLECTION, 1, 1);
-
-    SolrDispatchFilter solrDispatchFilter = cluster.getJettySolrRunner(0).getSolrDispatchFilter();
-
-    RateLimiterConfig queryRateLimiterConfig =
-        new RateLimiterConfig(
-            SolrRequest.SolrRequestType.QUERY,
-            true,
-            1,
-            DEFAULT_SLOT_ACQUISITION_TIMEOUT_MS,
-            5 /* allowedRequests */,
-            true /* isSlotBorrowing */);
-    RateLimiterConfig indexRateLimiterConfig =
-        new RateLimiterConfig(
-            SolrRequest.SolrRequestType.UPDATE,
-            true,
-            1,
-            DEFAULT_SLOT_ACQUISITION_TIMEOUT_MS,
-            5 /* allowedRequests */,
-            true /* isSlotBorrowing */);
-    // We are fine with a null FilterConfig here since we ensure that MockBuilder never invokes its
-    // parent
-    RateLimitManager.Builder builder =
-        new MockBuilder(
-            null /*dummy SolrZkClient */,
-            new MockRequestRateLimiter(queryRateLimiterConfig),
-            new MockRequestRateLimiter(indexRateLimiterConfig));
-    RateLimitManager rateLimitManager = builder.build();
-
-    solrDispatchFilter.replaceRateLimitManager(rateLimitManager);
-
-    int numDocs = 10000;
-
-    processTest(client, numDocs, 400 /* Number of queries */);
-
-    MockRequestRateLimiter mockIndexRateLimiter =
-        (MockRequestRateLimiter)
-            rateLimitManager.getRequestRateLimiter(SolrRequest.SolrRequestType.UPDATE);
-
-    assertTrue(
-        "Incoming slots borrowed count did not match. Expected > 0  incoming "
-            + mockIndexRateLimiter.borrowedSlotCount.get(),
-        mockIndexRateLimiter.borrowedSlotCount.get() > 0);
+    try (CloudSolrClient client =
+        cluster.basicSolrClientBuilder().withDefaultCollection(SECOND_COLLECTION).build()) {
+
+      CollectionAdminRequest.createCollection(SECOND_COLLECTION, 1, 1).process(client);
+      cluster.waitForActiveCollection(SECOND_COLLECTION, 1, 1);
+
+      SolrDispatchFilter solrDispatchFilter = cluster.getJettySolrRunner(0).getSolrDispatchFilter();
+
+      RateLimiterConfig queryRateLimiterConfig =
+          new RateLimiterConfig(
+              SolrRequest.SolrRequestType.QUERY,
+              true,
+              1,
+              DEFAULT_SLOT_ACQUISITION_TIMEOUT_MS,
+              5 /* allowedRequests */,
+              true /* isSlotBorrowing */);
+      RateLimiterConfig indexRateLimiterConfig =
+          new RateLimiterConfig(
+              SolrRequest.SolrRequestType.UPDATE,
+              true,
+              1,
+              DEFAULT_SLOT_ACQUISITION_TIMEOUT_MS,
+              5 /* allowedRequests */,
+              true /* isSlotBorrowing */);
+      // We are fine with a null FilterConfig here since we ensure that MockBuilder never invokes
+      // its parent
+      RateLimitManager.Builder builder =
+          new MockBuilder(
+              null /*dummy SolrZkClient */,
+              new MockRequestRateLimiter(queryRateLimiterConfig),
+              new MockRequestRateLimiter(indexRateLimiterConfig));
+      RateLimitManager rateLimitManager = builder.build();
+
+      solrDispatchFilter.replaceRateLimitManager(rateLimitManager);
+
+      int numDocs = 10000;
+
+      processTest(client, numDocs, 400 /* Number of queries */);
+
+      MockRequestRateLimiter mockIndexRateLimiter =
+          (MockRequestRateLimiter)
+              rateLimitManager.getRequestRateLimiter(SolrRequest.SolrRequestType.UPDATE);
+
+      assertTrue(
+          "Incoming slots borrowed count did not match. Expected > 0  incoming "
+              + mockIndexRateLimiter.borrowedSlotCount.get(),
+          mockIndexRateLimiter.borrowedSlotCount.get() > 0);
+    }
   }
 
-  private void processTest(CloudSolrClient client, int numDocuments, int numQueries)
-      throws Exception {
+  private void processTest(SolrClient client, int numDocuments, int numQueries) throws Exception {
 
     for (int i = 0; i < numDocuments; i++) {
       SolrInputDocument doc = new SolrInputDocument();
diff --git a/solr/core/src/test/org/apache/solr/update/DeleteByIdWithRouterFieldTest.java b/solr/core/src/test/org/apache/solr/update/DeleteByIdWithRouterFieldTest.java
index e4047c93a84..df32ef6783b 100644
--- a/solr/core/src/test/org/apache/solr/update/DeleteByIdWithRouterFieldTest.java
+++ b/solr/core/src/test/org/apache/solr/update/DeleteByIdWithRouterFieldTest.java
@@ -26,6 +26,7 @@ import java.util.stream.Collectors;
 import org.apache.lucene.tests.util.TestUtil;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.impl.LBSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
@@ -50,6 +51,7 @@ public class DeleteByIdWithRouterFieldTest extends SolrCloudTestCase {
   public static final int NUM_SHARDS = 3;
 
   private static final List<SolrClient> clients = new ArrayList<>(); // not CloudSolrClient
+  private static SolrClient solrClient;
 
   /**
    * A randomized prefix to put on every route value. This helps ensure that documents wind up on
@@ -75,11 +77,11 @@ public class DeleteByIdWithRouterFieldTest extends SolrCloudTestCase {
             .process(cluster.getSolrClient())
             .isSuccess());
 
-    cluster.getSolrClient().setDefaultCollection(COLL);
+    solrClient = cluster.basicSolrClientBuilder().withDefaultCollection(COLL).build();
 
     ClusterState clusterState = cluster.getSolrClient().getClusterState();
     for (Replica replica : clusterState.getCollection(COLL).getReplicas()) {
-      clients.add(getHttpSolrClient(replica.getCoreUrl()));
+      clients.add(new HttpSolrClient.Builder(replica.getCoreUrl()).build());
     }
   }
 
@@ -87,6 +89,8 @@ public class DeleteByIdWithRouterFieldTest extends SolrCloudTestCase {
   public static void afterClass() throws Exception {
     IOUtils.close(clients);
     clients.clear();
+    IOUtils.close(solrClient);
+
     RVAL_PRE = null;
   }
 
@@ -98,7 +102,7 @@ public class DeleteByIdWithRouterFieldTest extends SolrCloudTestCase {
         new UpdateRequest()
             .deleteByQuery("*:*")
             .setAction(UpdateRequest.ACTION.COMMIT, true, true)
-            .process(cluster.getSolrClient())
+            .process(solrClient)
             .getStatus());
   }
 
@@ -130,7 +134,7 @@ public class DeleteByIdWithRouterFieldTest extends SolrCloudTestCase {
 
   private SolrClient getRandomSolrClient() {
     final int index = random().nextInt(clients.size() + 1);
-    return index == clients.size() ? cluster.getSolrClient() : clients.get(index);
+    return index == clients.size() ? solrClient : clients.get(index);
   }
 
   /**
diff --git a/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java b/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java
index 855e85e0c54..1ce6666a856 100644
--- a/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java
+++ b/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java
@@ -32,8 +32,10 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import org.apache.commons.exec.DefaultExecutor;
 import org.apache.commons.exec.ExecuteResultHandler;
 import org.apache.lucene.tests.util.LuceneTestCase;
@@ -516,12 +518,12 @@ public class TestSolrCLIRunExample extends SolrTestCaseJ4 {
     }
 
     // index some docs - to verify all is good for both shards
-    CloudSolrClient cloudClient = null;
-
-    try {
-      cloudClient = getCloudSolrClient(executor.solrCloudCluster.getZkServer().getZkAddress());
-      cloudClient.connect();
-      cloudClient.setDefaultCollection(collectionName);
+    try (CloudSolrClient cloudClient =
+        new RandomizingCloudSolrClientBuilder(
+                Collections.singletonList(executor.solrCloudCluster.getZkServer().getZkAddress()),
+                Optional.empty())
+            .withDefaultCollection(collectionName)
+            .build()) {
 
       int numDocs = 10;
       for (int d = 0; d < numDocs; d++) {
@@ -542,13 +544,6 @@ public class TestSolrCLIRunExample extends SolrTestCaseJ4 {
                 + " collection but only found "
                 + qr.getResults().getNumFound());
       }
-    } finally {
-      if (cloudClient != null) {
-        try {
-          cloudClient.close();
-        } catch (Exception ignore) {
-        }
-      }
     }
 
     File node1SolrHome = new File(solrExampleDir, "cloud/node1/solr");
diff --git a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/HdfsCollectionsApiTest.java b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/HdfsCollectionsApiTest.java
index 1de75075b94..c2afd069585 100644
--- a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/HdfsCollectionsApiTest.java
+++ b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/HdfsCollectionsApiTest.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.lucene.tests.util.QuickPatchThreadsFilter;
 import org.apache.solr.SolrIgnoredThreadsFilter;
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
@@ -41,7 +42,7 @@ import org.junit.BeforeClass;
       BadHdfsThreadsFilter.class // hdfs currently leaks thread(s)
     })
 @ThreadLeakLingering(
-    linger = 1000) // Wait at least 1 second for Netty GlobalEventExecutor to shutdown
+    linger = 1000) // Wait at least 1 second for Netty GlobalEventExecutor to shut down
 public class HdfsCollectionsApiTest extends SolrCloudTestCase {
 
   private static MiniDFSCluster dfsCluster;
@@ -72,34 +73,35 @@ public class HdfsCollectionsApiTest extends SolrCloudTestCase {
   public void testDataDirIsNotReused() throws Exception {
     JettySolrRunner jettySolrRunner = cluster.getJettySolrRunner(0);
     String collection = "test";
-    cluster.getSolrClient().setDefaultCollection(collection);
     CollectionAdminRequest.createCollection(collection, "conf1", 1, 1)
         .setCreateNodeSet(jettySolrRunner.getNodeName())
         .process(cluster.getSolrClient());
     waitForState("", collection, clusterShape(1, 1));
-    cluster.getSolrClient().setDefaultCollection(collection);
-    cluster.getSolrClient().add(new SolrInputDocument("id", "1"));
-    cluster.getSolrClient().add(new SolrInputDocument("id", "2"));
-    cluster.getSolrClient().commit();
-    cluster.getSolrClient().add(new SolrInputDocument("id", "3"));
+    try (CloudSolrClient solrClient =
+        cluster.basicSolrClientBuilder().withDefaultCollection(collection).build()) {
+      solrClient.add(new SolrInputDocument("id", "1"));
+      solrClient.add(new SolrInputDocument("id", "2"));
+      solrClient.commit();
+      solrClient.add(new SolrInputDocument("id", "3"));
 
-    jettySolrRunner.stop();
-    waitForState(
-        "",
-        collection,
-        (liveNodes, collectionState) -> {
-          Replica replica = collectionState.getSlice("shard1").getReplicas().iterator().next();
-          return replica.getState() == Replica.State.DOWN;
-        });
-    CollectionAdminRequest.deleteCollection(collection).process(cluster.getSolrClient());
+      jettySolrRunner.stop();
+      waitForState(
+          "",
+          collection,
+          (liveNodes, collectionState) -> {
+            Replica replica = collectionState.getSlice("shard1").getReplicas().iterator().next();
+            return replica.getState() == Replica.State.DOWN;
+          });
+      CollectionAdminRequest.deleteCollection(collection).process(cluster.getSolrClient());
 
-    jettySolrRunner.start();
+      jettySolrRunner.start();
 
-    CollectionAdminRequest.createCollection(collection, "conf1", 1, 1)
-        .setCreateNodeSet(cluster.getJettySolrRunner(1).getNodeName())
-        .process(cluster.getSolrClient());
-    waitForState("", collection, clusterShape(1, 1));
-    QueryResponse response = cluster.getSolrClient().query(collection, new SolrQuery("*:*"));
-    assertEquals(0L, response.getResults().getNumFound());
+      CollectionAdminRequest.createCollection(collection, "conf1", 1, 1)
+          .setCreateNodeSet(cluster.getJettySolrRunner(1).getNodeName())
+          .process(cluster.getSolrClient());
+      waitForState("", collection, clusterShape(1, 1));
+      QueryResponse response = cluster.getSolrClient().query(collection, new SolrQuery("*:*"));
+      assertEquals(0L, response.getResults().getNumFound());
+    }
   }
 }
diff --git a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/HdfsWriteToMultipleCollectionsTest.java b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/HdfsWriteToMultipleCollectionsTest.java
index d68f9df6f9a..05626a298f0 100644
--- a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/HdfsWriteToMultipleCollectionsTest.java
+++ b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/HdfsWriteToMultipleCollectionsTest.java
@@ -21,7 +21,9 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -112,8 +114,11 @@ public class HdfsWriteToMultipleCollectionsTest extends AbstractBasicDistributed
     List<CloudSolrClient> cloudClients = new ArrayList<>();
     List<StoppableIndexingThread> threads = new ArrayList<>();
     for (int i = 0; i < cnt; i++) {
-      CloudSolrClient client = getCloudSolrClient(zkServer.getZkAddress());
-      client.setDefaultCollection(ACOLLECTION + i);
+      CloudSolrClient client =
+          new RandomizingCloudSolrClientBuilder(
+                  Collections.singletonList(zkServer.getZkAddress()), Optional.empty())
+              .withDefaultCollection(ACOLLECTION + i)
+              .build();
       cloudClients.add(client);
       StoppableIndexingThread indexThread =
           new StoppableIndexingThread(null, client, "1", true, docCount, 1, true);
diff --git a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/SharedFileSystemAutoReplicaFailoverTest.java b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/SharedFileSystemAutoReplicaFailoverTest.java
index ddce8a280d2..329cc7fee11 100644
--- a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/SharedFileSystemAutoReplicaFailoverTest.java
+++ b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/SharedFileSystemAutoReplicaFailoverTest.java
@@ -39,6 +39,7 @@ import org.apache.solr.SolrIgnoredThreadsFilter;
 import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
@@ -309,9 +310,9 @@ public class SharedFileSystemAutoReplicaFailoverTest extends AbstractFullDistrib
         fail("expected: " + expectedResultSize + ", actual: " + actualResultSize);
       }
       SolrParams queryAll = new SolrQuery("*:*");
-      cloudClient.setDefaultCollection(collection);
+      CloudSolrClient solrClient = this.getSolrClient(collection);
       try {
-        QueryResponse queryResponse = cloudClient.query(queryAll);
+        QueryResponse queryResponse = solrClient.query(queryAll);
         actualResultSize = queryResponse.getResults().getNumFound();
         if (expectedResultSize == actualResultSize) {
           return;
@@ -327,15 +328,16 @@ public class SharedFileSystemAutoReplicaFailoverTest extends AbstractFullDistrib
 
   private void addDocs(String collection, int numDocs, boolean commit)
       throws SolrServerException, IOException {
+    CloudSolrClient solrClient = this.getSolrClient(collection);
     for (int docId = 1; docId <= numDocs; docId++) {
       SolrInputDocument doc = new SolrInputDocument();
       doc.addField("id", docId);
       doc.addField("text", "shard" + docId % 5);
-      cloudClient.setDefaultCollection(collection);
-      cloudClient.add(doc);
+
+      solrClient.add(doc);
     }
     if (commit) {
-      cloudClient.commit();
+      solrClient.commit();
     }
   }
 
diff --git a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/StressHdfsTest.java b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/StressHdfsTest.java
index b437e77c30c..d27e4c79d12 100644
--- a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/StressHdfsTest.java
+++ b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/cloud/StressHdfsTest.java
@@ -36,6 +36,7 @@ import org.apache.lucene.tests.util.QuickPatchThreadsFilter;
 import org.apache.solr.SolrIgnoredThreadsFilter;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.cloud.AbstractBasicDistributedZkTestBase;
 import org.apache.solr.common.cloud.ClusterState;
@@ -174,11 +175,11 @@ public class StressHdfsTest extends AbstractBasicDistributedZkTestBase {
       assertNotNull(replica.getProperties().toString(), replica.get("ulogDir"));
     }
 
-    cloudClient.setDefaultCollection(DELETE_DATA_DIR_COLLECTION);
-    ZkStateReader.from(cloudClient).forceUpdateCollection(DELETE_DATA_DIR_COLLECTION);
+    CloudSolrClient solrClient = this.getSolrClient(DELETE_DATA_DIR_COLLECTION);
+
+    ZkStateReader.from(solrClient).forceUpdateCollection(DELETE_DATA_DIR_COLLECTION);
     for (int i = 1; i < nShards + 1; i++) {
-      ZkStateReader.from(cloudClient)
-          .getLeaderRetry(DELETE_DATA_DIR_COLLECTION, "shard" + i, 30000);
+      ZkStateReader.from(solrClient).getLeaderRetry(DELETE_DATA_DIR_COLLECTION, "shard" + i, 30000);
     }
 
     // collect the data dirs
@@ -210,14 +211,14 @@ public class StressHdfsTest extends AbstractBasicDistributedZkTestBase {
     }
 
     if (random().nextBoolean()) {
-      cloudClient.deleteByQuery("*:*");
-      cloudClient.commit();
+      solrClient.deleteByQuery("*:*");
+      solrClient.commit();
 
-      assertEquals(0, cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound());
+      assertEquals(0, solrClient.query(new SolrQuery("*:*")).getResults().getNumFound());
     }
 
-    cloudClient.commit();
-    cloudClient.query(new SolrQuery("*:*"));
+    solrClient.commit();
+    solrClient.query(new SolrQuery("*:*"));
 
     // delete collection
     ModifiableSolrParams params = new ModifiableSolrParams();
@@ -225,7 +226,7 @@ public class StressHdfsTest extends AbstractBasicDistributedZkTestBase {
     params.set("name", DELETE_DATA_DIR_COLLECTION);
     QueryRequest request = new QueryRequest(params);
     request.setPath("/admin/collections");
-    cloudClient.request(request);
+    solrClient.request(request);
 
     final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
     while (cloudClient.getClusterState().hasCollection(DELETE_DATA_DIR_COLLECTION)) {
diff --git a/solr/modules/ltr/src/test/org/apache/solr/ltr/TestLTROnSolrCloud.java b/solr/modules/ltr/src/test/org/apache/solr/ltr/TestLTROnSolrCloud.java
index 205a6db3686..ce5aa367d39 100644
--- a/solr/modules/ltr/src/test/org/apache/solr/ltr/TestLTROnSolrCloud.java
+++ b/solr/modules/ltr/src/test/org/apache/solr/ltr/TestLTROnSolrCloud.java
@@ -329,8 +329,6 @@ public class TestLTROnSolrCloud extends TestRerankBase {
     Path configDir = tmpSolrHome.resolve("collection1/conf");
     solrCluster.uploadConfigSet(configDir, "conf1");
 
-    solrCluster.getSolrClient().setDefaultCollection(COLLECTION);
-
     createCollection(COLLECTION, "conf1", numShards, numReplicas);
     indexDocuments(COLLECTION);
     for (JettySolrRunner solrRunner : solrCluster.getJettySolrRunners()) {
diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/enabling-ssl.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/enabling-ssl.adoc
index d90203eea65..4450d677866 100644
--- a/solr/solr-ref-guide/modules/deployment-guide/pages/enabling-ssl.adoc
+++ b/solr/solr-ref-guide/modules/deployment-guide/pages/enabling-ssl.adoc
@@ -436,8 +436,7 @@ System.setProperty("javax.net.ssl.trustStore", "/path/to/solr-ssl.keystore.p12")
 System.setProperty("javax.net.ssl.trustStorePassword", "secret");
 System.setProperty("javax.net.ssl.trustStoreType", "pkcs12");
 String zkHost = "127.0.0.1:2181";
-CloudSolrClient client = new CloudSolrClient.Builder(Collections.singletonList(zkHost),Optional.empty()).build();
-client.setDefaultCollection("mycollection");
+CloudSolrClient client = new CloudSolrClient.Builder(Collections.singletonList(zkHost),Optional.empty()).withDefaultCollection("mycollection").build();
 SolrInputDocument doc = new SolrInputDocument();
 doc.addField("id", "1234");
 doc.addField("name", "A lovely summer holiday");
diff --git a/solr/solr-ref-guide/modules/indexing-guide/examples/IndexingNestedDocuments.java b/solr/solr-ref-guide/modules/indexing-guide/examples/IndexingNestedDocuments.java
index efecb4ed928..1b7579ea689 100644
--- a/solr/solr-ref-guide/modules/indexing-guide/examples/IndexingNestedDocuments.java
+++ b/solr/solr-ref-guide/modules/indexing-guide/examples/IndexingNestedDocuments.java
@@ -41,6 +41,9 @@ import org.junit.BeforeClass;
 public class IndexingNestedDocuments extends SolrCloudTestCase {
   public static final String ANON_KIDS_CONFIG = "anon_kids_configset";
 
+  // This is the client that we configure to use in the example docs.
+  private static SolrClient clientUsedInSolrJExample;
+
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(1)
@@ -55,9 +58,9 @@ public class IndexingNestedDocuments extends SolrCloudTestCase {
     cluster.deleteAllCollections();
   }
 
-  /** Syntactic sugar so code snippet doesn't refer to test-framework specific method name */
+  /** Returns the already configured client for a specific example. */
   public static SolrClient getSolrClient() {
-    return cluster.getSolrClient();
+    return clientUsedInSolrJExample;
   }
 
   /**
@@ -66,89 +69,94 @@ public class IndexingNestedDocuments extends SolrCloudTestCase {
    */
   public void testIndexingAnonKids() throws Exception {
     final String collection = "test_anon";
+
     CollectionAdminRequest.createCollection(collection, ANON_KIDS_CONFIG, 1, 1)
         .setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
         .process(cluster.getSolrClient());
-    cluster.getSolrClient().setDefaultCollection(collection);
+
+    // configure the client with the default collection name, to simplify our example below.
+    IndexingNestedDocuments.clientUsedInSolrJExample =
+        cluster.basicSolrClientBuilder().withDefaultCollection(collection).build();
 
     //
-    // DO NOT MODIFY THESE EXAMPLE DOCS WITH OUT MAKING THE SAME CHANGES TO THE JSON AND XML
-    // EQUIVILENT EXAMPLES IN 'indexing-nested-documents.adoc'
+    // DO NOT MODIFY THESE EXAMPLE DOCS WITHOUT MAKING THE SAME CHANGES TO THE JSON AND XML
+    // EQUIVALENT EXAMPLES IN 'indexing-nested-documents.adoc'
     //
 
     // tag::anon-kids[]
-    final SolrClient client = getSolrClient();
-
-    final SolrInputDocument p1 = new SolrInputDocument();
-    p1.setField("id", "P11!prod");
-    p1.setField("type_s", "PRODUCT");
-    p1.setField("name_s", "Swingline Stapler");
-    p1.setField("description_t", "The Cadillac of office staplers ...");
-    {
-      final SolrInputDocument s1 = new SolrInputDocument();
-      s1.setField("id", "P11!S21");
-      s1.setField("type_s", "SKU");
-      s1.setField("color_s", "RED");
-      s1.setField("price_i", 42);
+    try (SolrClient client = getSolrClient()) {
+
+      final SolrInputDocument p1 = new SolrInputDocument();
+      p1.setField("id", "P11!prod");
+      p1.setField("type_s", "PRODUCT");
+      p1.setField("name_s", "Swingline Stapler");
+      p1.setField("description_t", "The Cadillac of office staplers ...");
       {
+        final SolrInputDocument s1 = new SolrInputDocument();
+        s1.setField("id", "P11!S21");
+        s1.setField("type_s", "SKU");
+        s1.setField("color_s", "RED");
+        s1.setField("price_i", 42);
+        {
+          final SolrInputDocument m1 = new SolrInputDocument();
+          m1.setField("id", "P11!D41");
+          m1.setField("type_s", "MANUAL");
+          m1.setField("name_s", "Red Swingline Brochure");
+          m1.setField("pages_i", 1);
+          m1.setField("content_t", "...");
+
+          s1.addChildDocument(m1);
+        }
+
+        final SolrInputDocument s2 = new SolrInputDocument();
+        s2.setField("id", "P11!S31");
+        s2.setField("type_s", "SKU");
+        s2.setField("color_s", "BLACK");
+        s2.setField("price_i", 3);
+
         final SolrInputDocument m1 = new SolrInputDocument();
-        m1.setField("id", "P11!D41");
+        m1.setField("id", "P11!D51");
         m1.setField("type_s", "MANUAL");
-        m1.setField("name_s", "Red Swingline Brochure");
+        m1.setField("name_s", "Quick Reference Guide");
         m1.setField("pages_i", 1);
-        m1.setField("content_t", "...");
+        m1.setField("content_t", "How to use your stapler ...");
 
-        s1.addChildDocument(m1);
-      }
+        final SolrInputDocument m2 = new SolrInputDocument();
+        m2.setField("id", "P11!D61");
+        m2.setField("type_s", "MANUAL");
+        m2.setField("name_s", "Warranty Details");
+        m2.setField("pages_i", 42);
+        m2.setField("content_t", "... lifetime guarantee ...");
 
-      final SolrInputDocument s2 = new SolrInputDocument();
-      s2.setField("id", "P11!S31");
-      s2.setField("type_s", "SKU");
-      s2.setField("color_s", "BLACK");
-      s2.setField("price_i", 3);
-
-      final SolrInputDocument m1 = new SolrInputDocument();
-      m1.setField("id", "P11!D51");
-      m1.setField("type_s", "MANUAL");
-      m1.setField("name_s", "Quick Reference Guide");
-      m1.setField("pages_i", 1);
-      m1.setField("content_t", "How to use your stapler ...");
-
-      final SolrInputDocument m2 = new SolrInputDocument();
-      m2.setField("id", "P11!D61");
-      m2.setField("type_s", "MANUAL");
-      m2.setField("name_s", "Warranty Details");
-      m2.setField("pages_i", 42);
-      m2.setField("content_t", "... lifetime guarantee ...");
-
-      p1.addChildDocuments(Arrays.asList(s1, s2, m1, m2));
-    }
+        p1.addChildDocuments(Arrays.asList(s1, s2, m1, m2));
+      }
 
-    client.add(p1);
-    // end::anon-kids[]
+      client.add(p1);
+      // end::anon-kids[]
 
-    client.commit();
+      client.commit();
 
-    final SolrDocumentList docs =
-        getSolrClient()
-            .query(
-                new SolrQuery("description_t:Cadillac")
-                    .set("fl", "*,[child parentFilter='type_s:PRODUCT']"))
-            .getResults();
+      final SolrDocumentList docs =
+          getSolrClient()
+              .query(
+                  new SolrQuery("description_t:Cadillac")
+                      .set("fl", "*,[child parentFilter='type_s:PRODUCT']"))
+              .getResults();
 
-    assertEquals(1, docs.getNumFound());
-    assertEquals("P11!prod", docs.get(0).getFieldValue("id"));
+      assertEquals(1, docs.getNumFound());
+      assertEquals("P11!prod", docs.get(0).getFieldValue("id"));
 
-    // [child] returns a flat list of all (anon) descendents
-    assertEquals(5, docs.get(0).getChildDocumentCount());
-    assertEquals(5, docs.get(0).getChildDocuments().size());
+      // [child] returns a flat list of all (anon) descendents
+      assertEquals(5, docs.get(0).getChildDocumentCount());
+      assertEquals(5, docs.get(0).getChildDocuments().size());
 
-    // flat list is depth first...
-    final SolrDocument red_stapler_brochure = docs.get(0).getChildDocuments().get(0);
-    assertEquals("P11!D41", red_stapler_brochure.getFieldValue("id"));
+      // flat list is depth first...
+      final SolrDocument red_stapler_brochure = docs.get(0).getChildDocuments().get(0);
+      assertEquals("P11!D41", red_stapler_brochure.getFieldValue("id"));
 
-    final SolrDocument red_stapler = docs.get(0).getChildDocuments().get(1);
-    assertEquals("P11!S21", red_stapler.getFieldValue("id"));
+      final SolrDocument red_stapler = docs.get(0).getChildDocuments().get(1);
+      assertEquals("P11!S21", red_stapler.getFieldValue("id"));
+    }
   }
 
   /**
@@ -158,128 +166,132 @@ public class IndexingNestedDocuments extends SolrCloudTestCase {
   public void testIndexingUsingNestPath() throws Exception {
     final String collection = "test_anon";
     CollectionAdminRequest.createCollection(collection, 1, 1).process(cluster.getSolrClient());
-    cluster.getSolrClient().setDefaultCollection(collection);
+
+    // configure the client with the default collection name, to simplify our example below.
+    IndexingNestedDocuments.clientUsedInSolrJExample =
+        cluster.basicSolrClientBuilder().withDefaultCollection(collection).build();
 
     //
-    // DO NOT MODIFY THESE EXAMPLE DOCS WITH OUT MAKING THE SAME CHANGES TO THE JSON AND XML
-    // EQUIVILENT EXAMPLES IN 'indexing-nested-documents.adoc'
+    // DO NOT MODIFY THESE EXAMPLE DOCS WITHOUT MAKING THE SAME CHANGES TO THE JSON AND XML
+    // EQUIVALENT EXAMPLES IN 'indexing-nested-documents.adoc'
     //
 
     // tag::nest-path[]
-    final SolrClient client = getSolrClient();
-
-    final SolrInputDocument p1 = new SolrInputDocument();
-    p1.setField("id", "P11!prod");
-    p1.setField("name_s", "Swingline Stapler");
-    p1.setField("description_t", "The Cadillac of office staplers ...");
-    {
-      final SolrInputDocument s1 = new SolrInputDocument();
-      s1.setField("id", "P11!S21");
-      s1.setField("color_s", "RED");
-      s1.setField("price_i", 42);
+    try (SolrClient client = getSolrClient()) {
+
+      final SolrInputDocument p1 = new SolrInputDocument();
+      p1.setField("id", "P11!prod");
+      p1.setField("name_s", "Swingline Stapler");
+      p1.setField("description_t", "The Cadillac of office staplers ...");
+      {
+        final SolrInputDocument s1 = new SolrInputDocument();
+        s1.setField("id", "P11!S21");
+        s1.setField("color_s", "RED");
+        s1.setField("price_i", 42);
+        {
+          final SolrInputDocument m1 = new SolrInputDocument();
+          m1.setField("id", "P11!D41");
+          m1.setField("name_s", "Red Swingline Brochure");
+          m1.setField("pages_i", 1);
+          m1.setField("content_t", "...");
+
+          s1.setField("manuals", m1);
+        }
+
+        final SolrInputDocument s2 = new SolrInputDocument();
+        s2.setField("id", "P11!S31");
+        s2.setField("color_s", "BLACK");
+        s2.setField("price_i", 3);
+
+        p1.setField("skus", Arrays.asList(s1, s2));
+      }
       {
         final SolrInputDocument m1 = new SolrInputDocument();
-        m1.setField("id", "P11!D41");
-        m1.setField("name_s", "Red Swingline Brochure");
+        m1.setField("id", "P11!D51");
+        m1.setField("name_s", "Quick Reference Guide");
         m1.setField("pages_i", 1);
-        m1.setField("content_t", "...");
+        m1.setField("content_t", "How to use your stapler ...");
 
-        s1.setField("manuals", m1);
-      }
-
-      final SolrInputDocument s2 = new SolrInputDocument();
-      s2.setField("id", "P11!S31");
-      s2.setField("color_s", "BLACK");
-      s2.setField("price_i", 3);
+        final SolrInputDocument m2 = new SolrInputDocument();
+        m2.setField("id", "P11!D61");
+        m2.setField("name_s", "Warranty Details");
+        m2.setField("pages_i", 42);
+        m2.setField("content_t", "... lifetime guarantee ...");
 
-      p1.setField("skus", Arrays.asList(s1, s2));
-    }
-    {
-      final SolrInputDocument m1 = new SolrInputDocument();
-      m1.setField("id", "P11!D51");
-      m1.setField("name_s", "Quick Reference Guide");
-      m1.setField("pages_i", 1);
-      m1.setField("content_t", "How to use your stapler ...");
-
-      final SolrInputDocument m2 = new SolrInputDocument();
-      m2.setField("id", "P11!D61");
-      m2.setField("name_s", "Warranty Details");
-      m2.setField("pages_i", 42);
-      m2.setField("content_t", "... lifetime guarantee ...");
-
-      p1.setField("manuals", Arrays.asList(m1, m2));
-    }
+        p1.setField("manuals", Arrays.asList(m1, m2));
+      }
 
-    final SolrInputDocument p2 = new SolrInputDocument();
-    p2.setField("id", "P22!prod");
-    p2.setField("name_s", "Mont Blanc Fountain Pen");
-    p2.setField("description_t", "A Premium Writing Instrument ...");
-    {
-      final SolrInputDocument s1 = new SolrInputDocument();
-      s1.setField("id", "P22!S22");
-      s1.setField("color_s", "RED");
-      s1.setField("price_i", 89);
+      final SolrInputDocument p2 = new SolrInputDocument();
+      p2.setField("id", "P22!prod");
+      p2.setField("name_s", "Mont Blanc Fountain Pen");
+      p2.setField("description_t", "A Premium Writing Instrument ...");
+      {
+        final SolrInputDocument s1 = new SolrInputDocument();
+        s1.setField("id", "P22!S22");
+        s1.setField("color_s", "RED");
+        s1.setField("price_i", 89);
+        {
+          final SolrInputDocument m1 = new SolrInputDocument();
+          m1.setField("id", "P22!D42");
+          m1.setField("name_s", "Red Mont Blanc Brochure");
+          m1.setField("pages_i", 1);
+          m1.setField("content_t", "...");
+
+          s1.setField("manuals", m1);
+        }
+
+        final SolrInputDocument s2 = new SolrInputDocument();
+        s2.setField("id", "P22!S32");
+        s2.setField("color_s", "BLACK");
+        s2.setField("price_i", 67);
+
+        p2.setField("skus", Arrays.asList(s1, s2));
+      }
       {
         final SolrInputDocument m1 = new SolrInputDocument();
-        m1.setField("id", "P22!D42");
-        m1.setField("name_s", "Red Mont Blanc Brochure");
-        m1.setField("pages_i", 1);
-        m1.setField("content_t", "...");
+        m1.setField("id", "P22!D52");
+        m1.setField("name_s", "How To Use A Pen");
+        m1.setField("pages_i", 42);
+        m1.setField("content_t", "Start by removing the cap ...");
 
-        s1.setField("manuals", m1);
+        p2.setField("manuals", m1);
       }
 
-      final SolrInputDocument s2 = new SolrInputDocument();
-      s2.setField("id", "P22!S32");
-      s2.setField("color_s", "BLACK");
-      s2.setField("price_i", 67);
-
-      p2.setField("skus", Arrays.asList(s1, s2));
-    }
-    {
-      final SolrInputDocument m1 = new SolrInputDocument();
-      m1.setField("id", "P22!D52");
-      m1.setField("name_s", "How To Use A Pen");
-      m1.setField("pages_i", 42);
-      m1.setField("content_t", "Start by removing the cap ...");
-
-      p2.setField("manuals", m1);
-    }
-
-    client.add(Arrays.asList(p1, p2));
-    // end::nest-path[]
+      client.add(Arrays.asList(p1, p2));
+      // end::nest-path[]
 
-    client.commit();
+      client.commit();
 
-    // Now a quick sanity check that the nest path is working properly...
+      // Now a quick sanity check that the nest path is working properly...
 
-    final SolrDocumentList docs =
-        getSolrClient()
-            .query(new SolrQuery("description_t:Writing").set("fl", "*,[child]"))
-            .getResults();
+      final SolrDocumentList docs =
+          getSolrClient()
+              .query(new SolrQuery("description_t:Writing").set("fl", "*,[child]"))
+              .getResults();
 
-    assertEquals(1, docs.getNumFound());
-    assertEquals("P22!prod", docs.get(0).getFieldValue("id"));
+      assertEquals(1, docs.getNumFound());
+      assertEquals("P22!prod", docs.get(0).getFieldValue("id"));
 
-    assertEquals(1, docs.get(0).getFieldValues("manuals").size());
-    assertEquals(
-        SolrDocument.class, docs.get(0).getFieldValues("manuals").iterator().next().getClass());
+      assertEquals(1, docs.get(0).getFieldValues("manuals").size());
+      assertEquals(
+          SolrDocument.class, docs.get(0).getFieldValues("manuals").iterator().next().getClass());
 
-    assertEquals(2, docs.get(0).getFieldValues("skus").size());
-    final List<Object> skus = new ArrayList<>(docs.get(0).getFieldValues("skus"));
+      assertEquals(2, docs.get(0).getFieldValues("skus").size());
+      final List<Object> skus = new ArrayList<>(docs.get(0).getFieldValues("skus"));
 
-    assertEquals(SolrDocument.class, skus.get(0).getClass());
-    assertEquals(SolrDocument.class, skus.get(1).getClass());
+      assertEquals(SolrDocument.class, skus.get(0).getClass());
+      assertEquals(SolrDocument.class, skus.get(1).getClass());
 
-    final SolrDocument red_pen = (SolrDocument) skus.get(0);
-    assertEquals("P22!S22", red_pen.getFieldValue("id"));
+      final SolrDocument red_pen = (SolrDocument) skus.get(0);
+      assertEquals("P22!S22", red_pen.getFieldValue("id"));
 
-    assertEquals(1, red_pen.getFieldValues("manuals").size());
-    assertEquals(
-        SolrDocument.class, red_pen.getFieldValues("manuals").iterator().next().getClass());
+      assertEquals(1, red_pen.getFieldValues("manuals").size());
+      assertEquals(
+          SolrDocument.class, red_pen.getFieldValues("manuals").iterator().next().getClass());
 
-    final SolrDocument red_pen_brochure =
-        (SolrDocument) red_pen.getFieldValues("manuals").iterator().next();
-    assertEquals("P22!D42", red_pen_brochure.getFieldValue("id"));
+      final SolrDocument red_pen_brochure =
+          (SolrDocument) red_pen.getFieldValues("manuals").iterator().next();
+      assertEquals("P22!D42", red_pen_brochure.getFieldValue("id"));
+    }
   }
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
index 632f2ab808f..7cbd38d08b8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
@@ -66,6 +66,7 @@ public class CloudHttp2SolrClient extends CloudSolrClient {
       this.myClient = builder.httpClient;
     }
     this.retryExpiryTimeNano = builder.retryExpiryTimeNano;
+    this.defaultCollection = builder.defaultCollection;
     if (builder.requestWriter != null) {
       this.myClient.requestWriter = builder.requestWriter;
     }
@@ -154,6 +155,8 @@ public class CloudHttp2SolrClient extends CloudSolrClient {
     private ResponseParser responseParser;
     private long retryExpiryTimeNano =
         TimeUnit.NANOSECONDS.convert(3, TimeUnit.SECONDS); // 3 seconds or 3 million nanos
+
+    private String defaultCollection;
     private long timeToLiveSeconds = 60;
     private int parallelCacheRefreshesLocks = 3;
 
@@ -332,6 +335,11 @@ public class CloudHttp2SolrClient extends CloudSolrClient {
       return this;
     }
 
+    /** Sets the default collection for request. */
+    public Builder withDefaultCollection(String collection) {
+      this.defaultCollection = collection;
+      return this;
+    }
     /**
      * Sets the cache ttl for DocCollection Objects cached.
      *
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudLegacySolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudLegacySolrClient.java
index 64db0045d41..3299236dd44 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudLegacySolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudLegacySolrClient.java
@@ -87,6 +87,7 @@ public class CloudLegacySolrClient extends CloudSolrClient {
       this.stateProvider = builder.stateProvider;
     }
     this.retryExpiryTimeNano = builder.retryExpiryTimeNano;
+    this.defaultCollection = builder.defaultCollection;
     this.collectionStateCache.timeToLiveMs =
         TimeUnit.MILLISECONDS.convert(builder.timeToLiveSeconds, TimeUnit.SECONDS);
     this.clientIsInternal = builder.httpClient == null;
@@ -200,6 +201,7 @@ public class CloudLegacySolrClient extends CloudSolrClient {
     protected boolean shardLeadersOnly = true;
     protected boolean directUpdatesToLeadersOnly = false;
     protected boolean parallelUpdates = true;
+    protected String defaultCollection;
     protected long retryExpiryTimeNano =
         TimeUnit.NANOSECONDS.convert(3, TimeUnit.SECONDS); // 3 seconds or 3 million nanos
     protected ClusterStateProvider stateProvider;
@@ -373,6 +375,12 @@ public class CloudLegacySolrClient extends CloudSolrClient {
       return this;
     }
 
+    /** Sets the default collection for request. */
+    public Builder withDefaultCollection(String collection) {
+      this.defaultCollection = collection;
+      return this;
+    }
+
     /** Create a {@link CloudLegacySolrClient} based on the provided configuration. */
     public CloudLegacySolrClient build() {
       if (stateProvider == null) {
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 a9700ec9a20..3308aa70dea 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
@@ -91,7 +91,7 @@ public abstract class CloudSolrClient extends SolrClient {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private volatile String defaultCollection;
+  protected volatile String defaultCollection;
   // no of times collection state to be reloaded if stale state error is received
   private static final int MAX_STALE_RETRIES =
       Integer.parseInt(System.getProperty("cloudSolrClientMaxStaleRetries", "5"));
@@ -342,7 +342,12 @@ public abstract class CloudSolrClient extends SolrClient {
     getLbClient().setRequestWriter(requestWriter);
   }
 
-  /** Sets the default collection for request */
+  /**
+   * Sets the default collection for request.
+   *
+   * @deprecated use {@link CloudHttp2SolrClient.Builder#withDefaultCollection(String)} instead
+   */
+  @Deprecated
   public void setDefaultCollection(String collection) {
     this.defaultCollection = collection;
   }
@@ -815,6 +820,7 @@ public abstract class CloudSolrClient extends SolrClient {
     } else if (collection == null) {
       collection = defaultCollection;
     }
+
     List<String> inputCollections =
         collection == null ? Collections.emptyList() : StrUtils.splitSmart(collection, ",", true);
     return requestWithRetryOnStaleState(request, 0, inputCollections);
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java
index bc4a4941890..4b2afacffd4 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java
@@ -303,8 +303,8 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
                 Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
             .sendUpdatesOnlyToShardLeaders()
             .withParallelUpdates(true)
+            .withDefaultCollection("routing_collection")
             .build()) {
-      threadedClient.setDefaultCollection("routing_collection");
       response = threadedClient.request(request);
       if (threadedClient.isDirectUpdatesToLeadersOnly()) {
         checkSingleServer(response);
@@ -622,14 +622,18 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
 
   @Test
   public void testNonRetryableRequests() throws Exception {
-    try (CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress())) {
+
+    try (CloudSolrClient client =
+        new RandomizingCloudSolrClientBuilder(
+                Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            .withDefaultCollection("foo")
+            .build()) {
       // important to have one replica on each node
       RequestStatusState state =
           CollectionAdminRequest.createCollection("foo", "conf", 1, NODE_COUNT)
               .processAndWait(client, 60);
       if (state == RequestStatusState.COMPLETED) {
         cluster.waitForActiveCollection("foo", 1, NODE_COUNT);
-        client.setDefaultCollection("foo");
 
         Map<String, String> adminPathToMbean = new HashMap<>(CommonParams.ADMIN_PATHS.size());
         adminPathToMbean.put(
@@ -693,8 +697,11 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
 
   @Test
   public void checkCollectionParameters() throws Exception {
-
-    try (CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress())) {
+    try (CloudSolrClient client =
+        new RandomizingCloudSolrClientBuilder(
+                Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            .withDefaultCollection("multicollection1")
+            .build()) {
 
       String async1 =
           CollectionAdminRequest.createCollection("multicollection1", "conf", 2, 1)
@@ -707,7 +714,6 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
       CollectionAdminRequest.waitForAsyncRequest(async2, client, TIMEOUT);
       cluster.waitForActiveCollection("multicollection1", 2, 2);
       cluster.waitForActiveCollection("multicollection2", 2, 2);
-      client.setDefaultCollection("multicollection1");
 
       List<SolrInputDocument> docs = new ArrayList<>(3);
       for (int i = 0; i < 3; i++) {
@@ -963,12 +969,11 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
                 Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
             .withParallelUpdates(true)
             .sendDirectUpdatesToAnyShardReplica()
+            .withDefaultCollection(COL)
             // don't let collection cache entries get expired, even on a slow machine...
             .withCollectionCacheTtl(Integer.MAX_VALUE)
             .build()) {
 
-      stale_client.setDefaultCollection(COL);
-
       // do a query to populate stale_client's cache...
       assertEquals(0, stale_client.query(new SolrQuery("*:*")).getResults().getNumFound());
 
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
index d033159bc75..bccfde52bc3 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
@@ -304,8 +304,8 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
                 Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
             .sendUpdatesOnlyToShardLeaders()
             .withParallelUpdates(true)
+            .withDefaultCollection("routing_collection")
             .build()) {
-      threadedClient.setDefaultCollection("routing_collection");
       response = threadedClient.request(request);
       if (threadedClient.isDirectUpdatesToLeadersOnly()) {
         checkSingleServer(response);
@@ -625,14 +625,17 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
   public void testNonRetryableRequests() throws Exception {
     String collection = getSaferTestName();
 
-    try (CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress())) {
+    try (CloudSolrClient client =
+        new RandomizingCloudSolrClientBuilder(
+                Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            .withDefaultCollection(collection)
+            .build()) {
       // important to have one replica on each node
       RequestStatusState state =
           CollectionAdminRequest.createCollection(collection, "conf", 1, NODE_COUNT)
               .processAndWait(client, 60);
       if (state == RequestStatusState.COMPLETED) {
         cluster.waitForActiveCollection(collection, 1, NODE_COUNT);
-        client.setDefaultCollection(collection);
 
         Map<String, String> adminPathToMbean = new HashMap<>(CommonParams.ADMIN_PATHS.size());
         adminPathToMbean.put(
@@ -697,7 +700,11 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
   @Test
   public void checkCollectionParameters() throws Exception {
 
-    try (CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress())) {
+    try (CloudSolrClient client =
+        new RandomizingCloudSolrClientBuilder(
+                Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            .withDefaultCollection("multicollection1")
+            .build()) {
 
       String async1 =
           CollectionAdminRequest.createCollection("multicollection1", "conf", 2, 1)
@@ -712,7 +719,6 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
       CollectionAdminRequest.waitForAsyncRequest(async2, client, TIMEOUT);
       cluster.waitForActiveCollection("multicollection1", 2, 2);
       cluster.waitForActiveCollection("multicollection2", 2, 2);
-      client.setDefaultCollection("multicollection1");
 
       List<SolrInputDocument> docs = new ArrayList<>(3);
       for (int i = 0; i < 3; i++) {
@@ -974,10 +980,9 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
             .withParallelUpdates(true)
             // don't let collection cache entries get expired, even on a slow machine...
             .withCollectionCacheTtl(Integer.MAX_VALUE)
+            .withDefaultCollection(COL)
             .build()) {
 
-      stale_client.setDefaultCollection(COL);
-
       // do a query to populate stale_client's cache...
       assertEquals(0, stale_client.query(new SolrQuery("*:*")).getResults().getNumFound());
 
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index b04fa944483..9eb0d1b9a9e 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -2676,50 +2676,11 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
         .build();
   }
 
-  /**
-   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient. Tests that do
-   * not wish to have any randomized behavior should use the {@link
-   * org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
-   */
-  public static CloudSolrClient getCloudSolrClient(String zkHost, boolean shardLeadersOnly) {
-    if (shardLeadersOnly) {
-      return new RandomizingCloudSolrClientBuilder(
-              Collections.singletonList(zkHost), Optional.empty())
-          .sendUpdatesOnlyToShardLeaders()
-          .build();
-    }
-    return new RandomizingCloudSolrClientBuilder(
-            Collections.singletonList(zkHost), Optional.empty())
-        .sendUpdatesToAnyReplica()
-        .build();
-  }
-
   public static RandomizingCloudSolrClientBuilder newCloudSolrClient(String zkHost) {
     return new RandomizingCloudSolrClientBuilder(
         Collections.singletonList(zkHost), Optional.empty());
   }
 
-  /**
-   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient. Tests that do
-   * not wish to have any randomized behavior should use the {@link
-   * org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
-   */
-  public static CloudSolrClient getCloudSolrClient(
-      String zkHost, boolean shardLeadersOnly, int socketTimeoutMillis) {
-    if (shardLeadersOnly) {
-      return new RandomizingCloudSolrClientBuilder(
-              Collections.singletonList(zkHost), Optional.empty())
-          .sendUpdatesOnlyToShardLeaders()
-          .withSocketTimeout(socketTimeoutMillis, TimeUnit.MILLISECONDS)
-          .build();
-    }
-    return new RandomizingCloudSolrClientBuilder(
-            Collections.singletonList(zkHost), Optional.empty())
-        .sendUpdatesToAnyReplica()
-        .withSocketTimeout(socketTimeoutMillis, TimeUnit.MILLISECONDS)
-        .build();
-  }
-
   /**
    * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient. Tests that do
    * not wish to have any randomized behavior should use the {@link
@@ -2727,72 +2688,23 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
    */
   public static CloudSolrClient getCloudSolrClient(
       String zkHost,
+      String defaultCollection,
       boolean shardLeadersOnly,
       int connectionTimeoutMillis,
       int socketTimeoutMillis) {
+    RandomizingCloudSolrClientBuilder builder =
+        new RandomizingCloudSolrClientBuilder(Collections.singletonList(zkHost), Optional.empty());
     if (shardLeadersOnly) {
-      return new RandomizingCloudSolrClientBuilder(
-              Collections.singletonList(zkHost), Optional.empty())
-          .sendUpdatesOnlyToShardLeaders()
-          .withConnectionTimeout(connectionTimeoutMillis, TimeUnit.MILLISECONDS)
-          .withSocketTimeout(socketTimeoutMillis, TimeUnit.MILLISECONDS)
-          .build();
-    }
-    return new RandomizingCloudSolrClientBuilder(
-            Collections.singletonList(zkHost), Optional.empty())
-        .sendUpdatesToAnyReplica()
-        .withConnectionTimeout(connectionTimeoutMillis, TimeUnit.MILLISECONDS)
-        .withSocketTimeout(socketTimeoutMillis, TimeUnit.MILLISECONDS)
-        .build();
-  }
-
-  /**
-   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient. Tests that do
-   * not wish to have any randomized behavior should use the {@link
-   * org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
-   */
-  public static CloudSolrClient getCloudSolrClient(
-      String zkHost, boolean shardLeadersOnly, HttpClient httpClient) {
-    if (shardLeadersOnly) {
-      return new RandomizingCloudSolrClientBuilder(
-              Collections.singletonList(zkHost), Optional.empty())
-          .withHttpClient(httpClient)
-          .sendUpdatesOnlyToShardLeaders()
-          .build();
+      builder.sendUpdatesOnlyToShardLeaders();
+    } else {
+      builder.sendUpdatesToAllReplicasInShard();
     }
-    return new RandomizingCloudSolrClientBuilder(
-            Collections.singletonList(zkHost), Optional.empty())
-        .withHttpClient(httpClient)
-        .sendUpdatesToAnyReplica()
-        .build();
-  }
-
-  /**
-   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient. Tests that do
-   * not wish to have any randomized behavior should use the {@link
-   * org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
-   */
-  public static CloudSolrClient getCloudSolrClient(
-      String zkHost,
-      boolean shardLeadersOnly,
-      HttpClient httpClient,
-      int connectionTimeoutMillis,
-      int socketTimeoutMillis) {
-    if (shardLeadersOnly) {
-      return new RandomizingCloudSolrClientBuilder(
-              Collections.singletonList(zkHost), Optional.empty())
-          .withHttpClient(httpClient)
-          .sendUpdatesOnlyToShardLeaders()
-          .withConnectionTimeout(connectionTimeoutMillis, TimeUnit.MILLISECONDS)
-          .withSocketTimeout(socketTimeoutMillis, TimeUnit.MILLISECONDS)
-          .build();
+    if (defaultCollection != null) {
+      builder.withDefaultCollection(defaultCollection);
     }
-    return new RandomizingCloudSolrClientBuilder(
-            Collections.singletonList(zkHost), Optional.empty())
-        .withHttpClient(httpClient)
-        .sendUpdatesToAnyReplica()
-        .withConnectionTimeout(connectionTimeoutMillis, TimeUnit.MILLISECONDS)
-        .withSocketTimeout(socketTimeoutMillis, TimeUnit.MILLISECONDS)
+    return builder
+        .withConnectionTimeout(connectionTimeoutMillis)
+        .withSocketTimeout(socketTimeoutMillis)
         .build();
   }
 
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZk2TestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZk2TestBase.java
index 362efbc63f5..899efc03dcb 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZk2TestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZk2TestBase.java
@@ -325,8 +325,6 @@ public abstract class AbstractBasicDistributedZk2TestBase extends AbstractFullDi
 
     query("q", "*:*", "sort", "n_tl1 desc");
 
-    cloudClient.setDefaultCollection(DEFAULT_COLLECTION);
-
     long numFound2 = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
 
     assertEquals(numFound1 + 1, numFound2);
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
index 5dfd05b91d1..6e5d086f84f 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
@@ -1589,22 +1589,20 @@ public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDis
     indexDoc("collection2", getDoc(id, "10000000"));
     indexDoc("collection2", getDoc(id, "10000001"));
     indexDoc("collection2", getDoc(id, "10000003"));
-    getCommonCloudSolrClient().setDefaultCollection("collection2");
-    getCommonCloudSolrClient().add(getDoc(id, "10000004"));
-    getCommonCloudSolrClient().setDefaultCollection(null);
+
+    getSolrClient("collection2").add(getDoc(id, "10000004"));
 
     indexDoc("collection3", getDoc(id, "20000000"));
     indexDoc("collection3", getDoc(id, "20000001"));
-    getCommonCloudSolrClient().setDefaultCollection("collection3");
-    getCommonCloudSolrClient().add(getDoc(id, "10000005"));
-    getCommonCloudSolrClient().setDefaultCollection(null);
+
+    getSolrClient("collection3").add(getDoc(id, "10000005"));
 
     otherCollectionClients.get("collection2").get(0).commit();
     otherCollectionClients.get("collection3").get(0).commit();
 
-    getCommonCloudSolrClient().setDefaultCollection("collection1");
+    SolrClient clientForCollection1 = getSolrClient("collection1");
     long collection1Docs =
-        getCommonCloudSolrClient().query(new SolrQuery("*:*")).getResults().getNumFound();
+        clientForCollection1.query(new SolrQuery("*:*")).getResults().getNumFound();
 
     long collection2Docs =
         otherCollectionClients
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractChaosMonkeyNothingIsSafeTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractChaosMonkeyNothingIsSafeTestBase.java
index b50a9f7854a..fb138876b30 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractChaosMonkeyNothingIsSafeTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractChaosMonkeyNothingIsSafeTestBase.java
@@ -132,10 +132,9 @@ public abstract class AbstractChaosMonkeyNothingIsSafeTestBase
   }
 
   protected CloudSolrClient createCloudClient(String defaultCollection, int socketTimeout) {
-    CloudSolrClient client =
-        getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean(), 30000, socketTimeout);
-    if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
-    return client;
+
+    return getCloudSolrClient(
+        zkServer.getZkAddress(), defaultCollection, random().nextBoolean(), 30000, socketTimeout);
   }
 
   @Test
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
index 5ff2914ffa3..0b3948d4fd6 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
@@ -147,6 +147,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   protected volatile ChaosMonkey chaosMonkey;
 
   protected Map<String, CloudJettyRunner> shardToLeaderJetty = new ConcurrentHashMap<>();
+  protected Map<String, CloudSolrClient> solrClientByCollection = new ConcurrentHashMap<>();
   private static volatile boolean cloudInit;
   protected volatile boolean useJettyDataDir = true;
 
@@ -331,10 +332,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   }
 
   protected CloudSolrClient createCloudClient(String defaultCollection) {
-    CloudSolrClient client =
-        getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean(), 30000, 120000);
-    if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
-    return client;
+    return getCloudSolrClient(
+        zkServer.getZkAddress(), defaultCollection, random().nextBoolean(), 30000, 120000);
   }
 
   @Override
@@ -800,7 +799,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   }
 
   /**
-   * Creates a JettySolrRunner with a socket proxy sitting infront of the Jetty server, which gives
+   * Creates a JettySolrRunner with a socket proxy sitting in front of the Jetty server, which gives
    * us the ability to simulate network partitions without having to fuss with IPTables.
    */
   public JettySolrRunner createProxiedJetty(
@@ -1030,10 +1029,14 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   }
 
   protected int sendDocsWithRetry(
-      List<SolrInputDocument> batch, int minRf, int maxRetries, int waitBeforeRetry)
+      String collectionName,
+      List<SolrInputDocument> batch,
+      int minRf,
+      int maxRetries,
+      int waitBeforeRetry)
       throws Exception {
     return sendDocsWithRetry(
-        cloudClient, cloudClient.getDefaultCollection(), batch, minRf, maxRetries, waitBeforeRetry);
+        cloudClient, collectionName, batch, minRf, maxRetries, waitBeforeRetry);
   }
 
   @SuppressWarnings("rawtypes")
@@ -1990,6 +1993,14 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
                       IOUtils.closeQuietly(c);
                     }));
 
+    customThreadPool.submit(
+        () ->
+            solrClientByCollection.values().parallelStream()
+                .forEach(
+                    c -> {
+                      IOUtils.closeQuietly(c);
+                    }));
+
     customThreadPool.submit(() -> IOUtils.closeQuietly(controlClientCloud));
 
     customThreadPool.submit(() -> IOUtils.closeQuietly(cloudClient));
@@ -1997,6 +2008,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
     ExecutorUtil.shutdownAndAwaitTermination(customThreadPool);
 
     coreClients.clear();
+    solrClientByCollection.clear();
 
     super.destroyServers();
   }
@@ -2269,8 +2281,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
     synchronized (this) {
       if (commonCloudSolrClient == null) {
         commonCloudSolrClient =
-            getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean(), 5000, 120000);
-        commonCloudSolrClient.setDefaultCollection(DEFAULT_COLLECTION);
+            getCloudSolrClient(
+                zkServer.getZkAddress(), DEFAULT_COLLECTION, random().nextBoolean(), 5000, 120000);
         commonCloudSolrClient.connect();
         if (log.isInfoEnabled()) {
           log.info(
@@ -2283,6 +2295,26 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
     return commonCloudSolrClient;
   }
 
+  protected CloudSolrClient getSolrClient(String collectionName) {
+    return solrClientByCollection.computeIfAbsent(
+        collectionName,
+        k -> {
+          CloudSolrClient solrClient =
+              getCloudSolrClient(
+                  zkServer.getZkAddress(), collectionName, random().nextBoolean(), 5000, 120000);
+
+          solrClient.connect();
+          if (log.isInfoEnabled()) {
+            log.info(
+                "Created solrClient for collection {} with updatesToLeaders={} and parallelUpdates={}",
+                collectionName,
+                solrClient.isUpdatesToLeaders(),
+                solrClient.isParallelUpdates());
+          }
+          return solrClient;
+        });
+  }
+
   public static String getUrlFromZk(ClusterState clusterState, String collection) {
     Map<String, Slice> slices = clusterState.getCollection(collection).getSlicesMap();
 
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractRecoveryZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractRecoveryZkTestBase.java
index 466cc7e5e43..c4975e388d2 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractRecoveryZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractRecoveryZkTestBase.java
@@ -20,6 +20,7 @@ import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
+import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.impl.CloudLegacySolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
@@ -41,7 +42,7 @@ public abstract class AbstractRecoveryZkTestBase extends SolrCloudTestCase {
 
   @BeforeClass
   public static void setupCluster() throws Exception {
-    configureCluster(2).addConfig("conf", configset("cloud-minimal")).configure();
+    cluster = configureCluster(2).addConfig("conf", configset("cloud-minimal")).configure();
   }
 
   private final List<StoppableIndexingThread> threads = new ArrayList<>();
@@ -66,7 +67,6 @@ public abstract class AbstractRecoveryZkTestBase extends SolrCloudTestCase {
         .process(cluster.getSolrClient());
     waitForState(
         "Expected a collection with one shard and two replicas", collection, clusterShape(1, 2));
-    cluster.getSolrClient().setDefaultCollection(collection);
 
     // start a couple indexing threads
 
@@ -81,51 +81,56 @@ public abstract class AbstractRecoveryZkTestBase extends SolrCloudTestCase {
     }
     log.info("Indexing {} documents", maxDoc);
 
-    final StoppableIndexingThread indexThread =
-        new StoppableIndexingThread(null, cluster.getSolrClient(), "1", true, maxDoc, 1, true);
-    threads.add(indexThread);
-    indexThread.start();
+    try (SolrClient solrClient =
+        cluster.basicSolrClientBuilder().withDefaultCollection(collection).build(); ) {
+      final StoppableIndexingThread indexThread =
+          new StoppableIndexingThread(null, solrClient, "1", true, maxDoc, 1, true);
+      threads.add(indexThread);
+      indexThread.start();
 
-    final StoppableIndexingThread indexThread2 =
-        new StoppableIndexingThread(null, cluster.getSolrClient(), "2", true, maxDoc, 1, true);
-    threads.add(indexThread2);
-    indexThread2.start();
+      final StoppableIndexingThread indexThread2 =
+          new StoppableIndexingThread(null, solrClient, "2", true, maxDoc, 1, true);
+      threads.add(indexThread2);
+      indexThread2.start();
 
-    // give some time to index...
-    int[] waitTimes = new int[] {200, 2000, 3000};
-    Thread.sleep(waitTimes[random().nextInt(waitTimes.length - 1)]);
+      // give some time to index...
+      int[] waitTimes = new int[] {200, 2000, 3000};
+      Thread.sleep(waitTimes[random().nextInt(waitTimes.length - 1)]);
 
-    // bring shard replica down
-    DocCollection state = getCollectionState(collection);
-    Replica leader = state.getLeader("shard1");
-    Replica replica = getRandomReplica(state.getSlice("shard1"), (r) -> !leader.equals(r));
+      // bring shard replica down
+      DocCollection state = getCollectionState(collection);
+      Replica leader = state.getLeader("shard1");
+      Replica replica = getRandomReplica(state.getSlice("shard1"), (r) -> !leader.equals(r));
 
-    JettySolrRunner jetty = cluster.getReplicaJetty(replica);
-    jetty.stop();
+      JettySolrRunner jetty = cluster.getReplicaJetty(replica);
+      jetty.stop();
 
-    // wait a moment - lets allow some docs to be indexed so replication time is non 0
-    Thread.sleep(waitTimes[random().nextInt(waitTimes.length - 1)]);
+      // wait a moment - lets allow some docs to be indexed so replication time is non 0
+      Thread.sleep(waitTimes[random().nextInt(waitTimes.length - 1)]);
 
-    // bring shard replica up
-    jetty.start();
+      // bring shard replica up
+      jetty.start();
 
-    // make sure replication can start
-    Thread.sleep(3000);
+      // make sure replication can start
+      Thread.sleep(3000);
 
-    // stop indexing threads
-    indexThread.safeStop();
-    indexThread2.safeStop();
+      // stop indexing threads
+      indexThread.safeStop();
+      indexThread2.safeStop();
 
-    indexThread.join();
-    indexThread2.join();
+      indexThread.join();
+      indexThread2.join();
 
-    new UpdateRequest().commit(cluster.getSolrClient(), collection);
+      new UpdateRequest().commit(solrClient, collection);
 
-    cluster.getZkStateReader().waitForState(collection, 120, TimeUnit.SECONDS, clusterShape(1, 2));
+      cluster
+          .getZkStateReader()
+          .waitForState(collection, 120, TimeUnit.SECONDS, clusterShape(1, 2));
 
-    // test that leader and replica have same doc count
-    state = getCollectionState(collection);
-    assertShardConsistency(state.getSlice("shard1"), true);
+      // test that leader and replica have same doc count
+      state = getCollectionState(collection);
+      assertShardConsistency(state.getSlice("shard1"), true);
+    }
   }
 
   private void assertShardConsistency(Slice shard, boolean expectDocs) throws Exception {
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 652faa02e37..cd9d35089eb 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
@@ -42,6 +42,7 @@ import java.util.Random;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
@@ -156,11 +157,12 @@ public class MiniSolrCloudCluster {
   private final boolean externalZkServer;
   private final List<JettySolrRunner> jettys = new CopyOnWriteArrayList<>();
   private final Path baseDir;
-  private final CloudSolrClient solrClient;
+  private CloudSolrClient solrClient;
   private final JettyConfig jettyConfig;
   private final boolean trackJettyMetrics;
 
   private final AtomicInteger nodeIds = new AtomicInteger();
+  private final Map<String, CloudSolrClient> solrClientByCollection = new ConcurrentHashMap<>();
 
   /**
    * Create a MiniSolrCloudCluster with default solr.xml
@@ -716,6 +718,14 @@ public class MiniSolrCloudCluster {
     try {
 
       IOUtils.closeQuietly(solrClient);
+
+      solrClientByCollection.values().parallelStream()
+          .forEach(
+              c -> {
+                IOUtils.closeQuietly(c);
+              });
+      solrClientByCollection.clear();
+      ;
       List<Callable<JettySolrRunner>> shutdowns = new ArrayList<>(jettys.size());
       for (final JettySolrRunner jetty : jettys) {
         shutdowns.add(() -> stopJettySolrRunner(jetty));
@@ -745,6 +755,37 @@ public class MiniSolrCloudCluster {
     return solrClient;
   }
 
+  /**
+   * Returns a SolrClient that has a defaultCollection set for it. SolrClients are cached by their
+   * collectionName for reuse and are closed for you.
+   *
+   * @param collectionName The name of the collection to get a SolrClient for.
+   * @return CloudSolrClient configured for the specific collection.
+   */
+  public CloudSolrClient getSolrClient(String collectionName) {
+    return solrClientByCollection.computeIfAbsent(
+        collectionName,
+        k -> {
+          CloudSolrClient solrClient =
+              new CloudLegacySolrClient.Builder(
+                      Collections.singletonList(zkServer.getZkAddress()), Optional.empty())
+                  .withDefaultCollection(collectionName)
+                  .withSocketTimeout(90000)
+                  .withConnectionTimeout(15000)
+                  .build();
+
+          solrClient.connect();
+          if (log.isInfoEnabled()) {
+            log.info(
+                "Created solrClient for collection {} with updatesToLeaders={} and parallelUpdates={}",
+                collectionName,
+                solrClient.isUpdatesToLeaders(),
+                solrClient.isParallelUpdates());
+          }
+          return solrClient;
+        });
+  }
+
   public SolrZkClient getZkClient() {
     return getZkStateReader().getZkClient();
   }
@@ -770,6 +811,19 @@ public class MiniSolrCloudCluster {
         .build(); // we choose 90 because we run in some harsh envs
   }
 
+  /**
+   * creates a basic CloudSolrClient Builder that then can be customized by callers, for example by
+   * specifying what collection they want to use.
+   *
+   * @return CloudLegacySolrClient.Builder
+   */
+  public CloudLegacySolrClient.Builder basicSolrClientBuilder() {
+    return new CloudLegacySolrClient.Builder(
+            Collections.singletonList(getZkServer().getZkAddress()), Optional.empty())
+        .withSocketTimeout(90000) // we choose 90 because we run in some harsh envs
+        .withConnectionTimeout(15000);
+  }
+
   private static String getHostContextSuitableForServletContext(String ctx) {
     if (ctx == null || "".equals(ctx)) ctx = "/solr";
     if (ctx.endsWith("/")) ctx = ctx.substring(0, ctx.length() - 1);