You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ge...@apache.org on 2024/01/14 01:43:56 UTC

(solr) branch main updated: SOLR-17066: Switch HttpSolrClient away from core URLs, pt 1 (#2173)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 8e9c513f4a6 SOLR-17066: Switch HttpSolrClient away from core URLs, pt 1 (#2173)
8e9c513f4a6 is described below

commit 8e9c513f4a675d7f356d14781630f2c8063edf5f
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Sat Jan 13 20:43:51 2024 -0500

    SOLR-17066: Switch HttpSolrClient away from core URLs, pt 1 (#2173)
    
    Providing a core URL as a SolrClient's "base URL" prevents it from
    communicating with other cores or making core-agnostic API requests
    (e.g. node healthcheck, list cores, etc.)
    
    This commit migrates production (i.e. non-test) code in the 'core'
    module to use the `withDefaultCollection` builder method in lieu of
    using core-based URLs directly.
---
 .../org/apache/solr/cloud/RecoveryStrategy.java    | 34 +++++++-----
 .../api/collections/CollectionHandlingUtils.java   |  9 ++--
 .../api/collections/ReindexCollectionCmd.java      | 44 +++++++++------
 .../java/org/apache/solr/handler/IndexFetcher.java | 54 +++++++++++--------
 .../apache/solr/handler/ReplicationHandler.java    |  4 +-
 .../org/apache/solr/handler/SolrConfigHandler.java | 30 ++++++-----
 .../handler/component/IterativeMergeStrategy.java  | 10 +++-
 .../org/apache/solr/schema/ManagedIndexSchema.java | 36 +++++++------
 .../org/apache/solr/update/PeerSyncWithLeader.java | 10 +++-
 .../apache/solr/update/StreamingSolrClients.java   |  7 ++-
 .../solr/core/ConfigureRecoveryStrategyTest.java   |  4 +-
 .../apache/solr/update/SolrCmdDistributorTest.java | 27 +++++-----
 .../solrj/impl/BaseHttpClusterStateProvider.java   |  1 +
 .../solr/client/solrj/impl/HttpSolrClient.java     |  2 +-
 .../java/org/apache/solr/common/util/URLUtil.java  | 37 +++++++++++++
 .../solrj/response/TestSuggesterResponse.java      |  3 +-
 .../org/apache/solr/common/util/URLUtilTest.java   | 63 ++++++++++++++++++++++
 .../apache/solr/BaseDistributedSearchTestCase.java | 11 +---
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   |  8 +++
 19 files changed, 278 insertions(+), 116 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index 2c0ae4ea9be..bb5d584f9e8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -48,6 +48,7 @@ import org.apache.solr.common.cloud.ZooKeeperException;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.URLUtil;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.DirectoryFactory.DirContext;
@@ -175,11 +176,12 @@ public class RecoveryStrategy implements Runnable, Closeable {
   }
 
   /** Builds a new HttpSolrClient for use in recovery. Caller must close */
-  private final HttpSolrClient.Builder recoverySolrClientBuilder(final String leaderUrl) {
+  private HttpSolrClient.Builder recoverySolrClientBuilder(String baseUrl, String leaderCoreName) {
     // workaround for SOLR-13605: get the configured timeouts & set them directly
     // (even though getRecoveryOnlyHttpClient() already has them set)
     final UpdateShardHandlerConfig cfg = cc.getConfig().getUpdateShardHandlerConfig();
-    return (new HttpSolrClient.Builder(leaderUrl)
+    return (new HttpSolrClient.Builder(baseUrl)
+        .withDefaultDataStore(leaderCoreName)
         .withConnectionTimeout(cfg.getDistributedConnectionTimeout(), TimeUnit.MILLISECONDS)
         .withSocketTimeout(cfg.getDistributedSocketTimeout(), TimeUnit.MILLISECONDS)
         .withHttpClient(cc.getUpdateShardHandler().getRecoveryOnlyHttpClient()));
@@ -216,16 +218,17 @@ public class RecoveryStrategy implements Runnable, Closeable {
     return new ZkCoreNodeProps(leaderprops).getCoreUrl();
   }
 
-  private final void replicate(String nodeName, SolrCore core, ZkNodeProps leaderprops)
+  private void replicate(String nodeName, SolrCore core, ZkNodeProps leaderprops)
       throws SolrServerException, IOException {
 
-    final String leaderUrl = getReplicateLeaderUrl(leaderprops);
+    final String leaderBaseUrl = URLUtil.extractBaseUrl(getReplicateLeaderUrl(leaderprops));
+    final String leaderCore = URLUtil.extractCoreFromCoreUrl(getReplicateLeaderUrl(leaderprops));
 
-    log.info("Attempting to replicate from [{}].", leaderUrl);
+    log.info("Attempting to replicate from core [{}] on node [{}].", leaderCore, leaderBaseUrl);
 
     // send commit if replica could be a leader
     if (replicaType.leaderEligible) {
-      commitOnLeader(leaderUrl);
+      commitOnLeader(leaderBaseUrl, leaderCore);
     }
 
     // use rep handler directly, so we can do this sync rather than async
@@ -239,7 +242,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
     }
 
     ModifiableSolrParams solrParams = new ModifiableSolrParams();
-    solrParams.set(ReplicationHandler.LEADER_URL, leaderUrl);
+    solrParams.set(ReplicationHandler.LEADER_URL, URLUtil.buildCoreUrl(leaderBaseUrl, leaderCore));
     solrParams.set(
         ReplicationHandler.SKIP_COMMIT_ON_LEADER_VERSION_ZERO, replicaType == Replica.Type.TLOG);
 
@@ -261,10 +264,11 @@ public class RecoveryStrategy implements Runnable, Closeable {
           final IndexCommit commit = core.getDeletionPolicy().getLatestCommit();
           if (log.isDebugEnabled()) {
             log.debug(
-                "{} replicated {} from {} gen: {} data: {} index: {} newIndex: {} files: {}",
+                "{} replicated {} from core {} on node {} gen: {} data: {} index: {} newIndex: {} files: {}",
                 core.getCoreContainer().getZkController().getNodeName(),
                 searcher.count(new MatchAllDocsQuery()),
-                leaderUrl,
+                leaderCore,
+                leaderBaseUrl,
                 (null == commit ? "null" : commit.getGeneration()),
                 core.getDataDir(),
                 core.getIndexDir(),
@@ -281,8 +285,9 @@ public class RecoveryStrategy implements Runnable, Closeable {
     }
   }
 
-  private final void commitOnLeader(String leaderUrl) throws SolrServerException, IOException {
-    try (SolrClient client = recoverySolrClientBuilder(leaderUrl).build()) {
+  private void commitOnLeader(String leaderBaseUrl, String coreName)
+      throws SolrServerException, IOException {
+    try (SolrClient client = recoverySolrClientBuilder(leaderBaseUrl, coreName).build()) {
       UpdateRequest ureq = new UpdateRequest();
       ureq.setParams(new ModifiableSolrParams());
       // ureq.getParams().set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
@@ -801,7 +806,8 @@ public class RecoveryStrategy implements Runnable, Closeable {
       }
 
       try (SolrClient httpSolrClient =
-          recoverySolrClientBuilder(leaderReplica.getCoreUrl()).build()) {
+          recoverySolrClientBuilder(leaderReplica.getBaseUrl(), leaderReplica.getCoreName())
+              .build()) {
         httpSolrClient.ping();
         return leaderReplica;
       } catch (IOException e) {
@@ -906,7 +912,9 @@ public class RecoveryStrategy implements Runnable, Closeable {
         conflictWaitMs
             + Integer.parseInt(System.getProperty("prepRecoveryReadTimeoutExtraWait", "8000"));
     try (HttpSolrClient client =
-        recoverySolrClientBuilder(leaderBaseUrl)
+        recoverySolrClientBuilder(
+                leaderBaseUrl,
+                null) // leader core omitted since client only used for 'admin' request
             .withSocketTimeout(readTimeout, TimeUnit.MILLISECONDS)
             .build()) {
       HttpUriRequestResponse mrr = client.httpUriRequest(prepCmd);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CollectionHandlingUtils.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CollectionHandlingUtils.java
index f2462430f48..d9531049d9a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CollectionHandlingUtils.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CollectionHandlingUtils.java
@@ -236,12 +236,13 @@ public class CollectionHandlingUtils {
 
   static void commit(NamedList<Object> results, String slice, Replica parentShardLeader) {
     log.debug("Calling soft commit to make sub shard updates visible");
+    final var zkCoreProps = new ZkCoreNodeProps(parentShardLeader);
     String coreUrl = new ZkCoreNodeProps(parentShardLeader).getCoreUrl();
     // HttpShardHandler is hard coded to send a QueryRequest hence we go direct
     // and we force open a searcher so that we have documents to show upon switching states
     UpdateResponse updateResponse = null;
     try {
-      updateResponse = softCommit(coreUrl);
+      updateResponse = softCommit(zkCoreProps.getBaseUrl(), zkCoreProps.getCoreName());
       CollectionHandlingUtils.processResponse(
           results, null, coreUrl, updateResponse, slice, Collections.emptySet());
     } catch (Exception e) {
@@ -254,10 +255,12 @@ public class CollectionHandlingUtils {
     }
   }
 
-  static UpdateResponse softCommit(String url) throws SolrServerException, IOException {
+  static UpdateResponse softCommit(String baseUrl, String coreName)
+      throws SolrServerException, IOException {
 
     try (SolrClient client =
-        new HttpSolrClient.Builder(url)
+        new HttpSolrClient.Builder(baseUrl)
+            .withDefaultDataStore(coreName)
             .withConnectionTimeout(30000, TimeUnit.MILLISECONDS)
             .withSocketTimeout(120000, TimeUnit.MILLISECONDS)
             .build()) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
index 8a083d375cc..6f682e161d4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
@@ -272,6 +272,7 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
     }
     String chkCollection = CHK_COL_PREFIX + extCollection;
     String daemonUrl = null;
+    Replica daemonReplica = null;
     Exception exc = null;
     boolean createdTarget = false;
     try {
@@ -458,8 +459,8 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
             "Unable to copy documents from " + collection + " to " + targetCollection,
             e);
       }
-      daemonUrl = getDaemonUrl(rsp, coll);
-      if (daemonUrl == null) {
+      daemonReplica = getReplicaForDaemon(rsp, coll);
+      if (daemonReplica == null) {
         throw new SolrException(
             SolrException.ErrorCode.SERVER_ERROR,
             "Unable to copy documents from "
@@ -469,13 +470,13 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
                 + ": "
                 + Utils.toJSONString(rsp));
       }
-      reindexingState.put("daemonUrl", daemonUrl);
+      reindexingState.put("daemonUrl", daemonReplica.getCoreUrl());
       reindexingState.put("daemonName", targetCollection);
       reindexingState.put(PHASE, "copying documents");
       setReindexingState(collection, State.RUNNING, reindexingState);
 
       // wait for the daemon to finish
-      waitForDaemon(targetCollection, daemonUrl, collection, targetCollection, reindexingState);
+      waitForDaemon(targetCollection, daemonReplica, collection, targetCollection, reindexingState);
       if (maybeAbort(collection)) {
         aborted = true;
         return;
@@ -587,7 +588,7 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
             collection,
             targetCollection,
             chkCollection,
-            daemonUrl,
+            daemonReplica,
             targetCollection,
             createdTarget);
         if (exc != null) {
@@ -667,7 +668,7 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
   }
 
   // XXX see #waitForDaemon() for why we need this
-  private String getDaemonUrl(SolrResponse rsp, DocCollection coll) {
+  private Replica getReplicaForDaemon(SolrResponse rsp, DocCollection coll) {
     @SuppressWarnings({"unchecked"})
     Map<String, Object> rs = (Map<String, Object>) rsp.getResponse().get("result-set");
     if (rs == null || rs.isEmpty()) {
@@ -711,7 +712,7 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
     // build a baseUrl of the replica
     for (Replica r : coll.getReplicas()) {
       if (replicaName.equals(r.getCoreName())) {
-        return r.getBaseUrl() + "/" + r.getCoreName();
+        return r;
       }
     }
     return null;
@@ -723,14 +724,17 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
   @SuppressWarnings({"unchecked"})
   private void waitForDaemon(
       String daemonName,
-      String daemonUrl,
+      Replica daemonReplica,
       String sourceCollection,
       String targetCollection,
       Map<String, Object> reindexingState)
       throws Exception {
     HttpClient client = ccc.getCoreContainer().getUpdateShardHandler().getDefaultHttpClient();
     try (SolrClient solrClient =
-        new HttpSolrClient.Builder().withHttpClient(client).withBaseSolrUrl(daemonUrl).build()) {
+        new HttpSolrClient.Builder()
+            .withHttpClient(client)
+            .withBaseSolrUrl(daemonReplica.getBaseUrl())
+            .build()) {
       ModifiableSolrParams q = new ModifiableSolrParams();
       q.set(CommonParams.QT, "/stream");
       q.set("action", "list");
@@ -742,7 +746,7 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
         isRunning = false;
         statusCheck++;
         try {
-          NamedList<Object> rsp = solrClient.request(req);
+          NamedList<Object> rsp = solrClient.request(req, daemonReplica.getCoreName());
           Map<String, Object> rs = (Map<String, Object>) rsp.get("result-set");
           if (rs == null || rs.isEmpty()) {
             throw new SolrException(
@@ -771,7 +775,7 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
         } catch (Exception e) {
           throw new SolrException(
               SolrException.ErrorCode.SERVER_ERROR,
-              "Exception waiting for daemon " + daemonName + " at " + daemonUrl,
+              "Exception waiting for daemon " + daemonName + " at " + daemonReplica.getCoreUrl(),
               e);
         }
         if (statusCheck % 5 == 0) {
@@ -784,11 +788,17 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
   }
 
   @SuppressWarnings({"unchecked"})
-  private void killDaemon(String daemonName, String daemonUrl) throws Exception {
-    log.debug("-- killing daemon {} at {}", daemonName, daemonUrl);
+  private void killDaemon(String daemonName, Replica daemonReplica) throws Exception {
+    if (log.isDebugEnabled()) {
+      log.debug("-- killing daemon {} at {}", daemonName, daemonReplica.getCoreUrl());
+    }
     HttpClient client = ccc.getCoreContainer().getUpdateShardHandler().getDefaultHttpClient();
     try (SolrClient solrClient =
-        new HttpSolrClient.Builder().withHttpClient(client).withBaseSolrUrl(daemonUrl).build()) {
+        new HttpSolrClient.Builder()
+            .withHttpClient(client)
+            .withDefaultDataStore(daemonReplica.getCoreName())
+            .withBaseSolrUrl(daemonReplica.getBaseUrl())
+            .build()) {
       ModifiableSolrParams q = new ModifiableSolrParams();
       q.set(CommonParams.QT, "/stream");
       // we should really use 'kill' here, but then we will never
@@ -889,7 +899,7 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
       String collection,
       String targetCollection,
       String chkCollection,
-      String daemonUrl,
+      Replica daemonReplica,
       String daemonName,
       boolean createdTarget)
       throws Exception {
@@ -898,8 +908,8 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
     // 2. cleanup target / chk collections IFF the source collection still exists and is not empty
     // 3. cleanup collection state
 
-    if (daemonUrl != null) {
-      killDaemon(daemonName, daemonUrl);
+    if (daemonReplica != null) {
+      killDaemon(daemonName, daemonReplica);
     }
     ClusterState clusterState = ccc.getSolrCloudManager().getClusterState();
     NamedList<Object> cmdResults = new NamedList<>();
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index 1df9d1e101c..4dbe9faf44b 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -116,6 +116,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.SuppressForbidden;
+import org.apache.solr.common.util.URLUtil;
 import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
@@ -149,7 +150,10 @@ public class IndexFetcher {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private String leaderUrl;
+  private String leaderCoreUrl;
+  // Derived from 'leaderCoreUrl' but kept around to avoid recomputing
+  private String leaderBaseUrl;
+  private String leaderCoreName;
 
   final ReplicationHandler replicationHandler;
 
@@ -300,7 +304,7 @@ public class IndexFetcher {
       leaderUrl = leaderUrl.substring(0, leaderUrl.length() - 12);
       log.warn("'leaderUrl' must be specified without the {} suffix", ReplicationHandler.PATH);
     }
-    setLeaderUrl(leaderUrl);
+    setLeaderCoreUrl(leaderUrl);
 
     this.replicationHandler = handler;
     String compress = (String) initArgs.get(COMPRESSION);
@@ -322,8 +326,8 @@ public class IndexFetcher {
             solrCore, httpBasicAuthUser, httpBasicAuthPassword, useExternalCompression);
   }
 
-  private void setLeaderUrl(String leaderUrl) {
-    if (leaderUrl != null) {
+  private void setLeaderCoreUrl(String leaderCoreUrl) {
+    if (leaderCoreUrl != null) {
       ClusterState clusterState =
           solrCore.getCoreContainer().getZkController() == null
               ? null
@@ -332,24 +336,28 @@ public class IndexFetcher {
         solrCore
             .getCoreContainer()
             .getAllowListUrlChecker()
-            .checkAllowList(Collections.singletonList(leaderUrl), clusterState);
+            .checkAllowList(Collections.singletonList(leaderCoreUrl), clusterState);
       } catch (MalformedURLException e) {
         throw new SolrException(
-            SolrException.ErrorCode.SERVER_ERROR, "Malformed 'leaderUrl' " + leaderUrl, e);
+            SolrException.ErrorCode.SERVER_ERROR, "Malformed 'leaderUrl' " + leaderCoreUrl, e);
       } catch (SolrException e) {
         throw new SolrException(
             SolrException.ErrorCode.FORBIDDEN,
             "The '"
                 + LEADER_URL
                 + "' parameter value '"
-                + leaderUrl
+                + leaderCoreUrl
                 + "' is not allowed: "
                 + e.getMessage()
                 + ". "
                 + AllowListUrlChecker.SET_SOLR_DISABLE_URL_ALLOW_LIST_CLUE);
       }
     }
-    this.leaderUrl = leaderUrl;
+    this.leaderCoreUrl = leaderCoreUrl;
+    if (leaderCoreUrl != null) {
+      this.leaderBaseUrl = URLUtil.extractBaseUrl(leaderCoreUrl);
+      this.leaderCoreName = URLUtil.extractCoreFromCoreUrl(leaderCoreUrl);
+    }
   }
 
   protected <T> T getParameter(
@@ -375,13 +383,13 @@ public class IndexFetcher {
 
     // TODO modify to use shardhandler
     try (SolrClient client =
-        new Builder(leaderUrl)
+        new Builder(leaderBaseUrl)
             .withHttpClient(myHttpClient)
             .withConnectionTimeout(connTimeout, TimeUnit.MILLISECONDS)
             .withSocketTimeout(soTimeout, TimeUnit.MILLISECONDS)
             .build()) {
 
-      return client.request(req);
+      return client.request(req, leaderCoreName);
     } catch (SolrServerException e) {
       throw new SolrException(ErrorCode.SERVER_ERROR, e.getMessage(), e);
     }
@@ -402,12 +410,12 @@ public class IndexFetcher {
 
     // TODO modify to use shardhandler
     try (SolrClient client =
-        new HttpSolrClient.Builder(leaderUrl)
+        new HttpSolrClient.Builder(leaderBaseUrl)
             .withHttpClient(myHttpClient)
             .withConnectionTimeout(connTimeout, TimeUnit.MILLISECONDS)
             .withSocketTimeout(soTimeout, TimeUnit.MILLISECONDS)
             .build()) {
-      NamedList<?> response = client.request(req);
+      NamedList<?> response = client.request(req, leaderCoreName);
 
       List<Map<String, Object>> files = (List<Map<String, Object>>) response.get(CMD_GET_FILE_LIST);
       if (files != null) filesToDownload = Collections.synchronizedList(files);
@@ -488,9 +496,9 @@ public class IndexFetcher {
           }
           return IndexFetchResult.LEADER_IS_NOT_ACTIVE;
         }
-        if (!replica.getCoreUrl().equals(leaderUrl)) {
-          setLeaderUrl(replica.getCoreUrl());
-          log.info("Updated leaderUrl to {}", leaderUrl);
+        if (!replica.getCoreUrl().equals(leaderCoreUrl)) {
+          setLeaderCoreUrl(replica.getCoreUrl());
+          log.info("Updated leaderUrl to {}", leaderCoreUrl);
           // TODO: Do we need to set forceReplication = true?
         } else {
           log.debug("leaderUrl didn't change");
@@ -505,13 +513,13 @@ public class IndexFetcher {
         if (StrUtils.isNotNullOrEmpty(errorMsg) && errorMsg.contains(INTERRUPT_RESPONSE_MESSAGE)) {
           log.warn(
               "Leader at: {} is not available. Index fetch failed by interrupt. Exception: {}",
-              leaderUrl,
+              leaderCoreUrl,
               errorMsg);
           return new IndexFetchResult(IndexFetchResult.FAILED_BY_INTERRUPT_MESSAGE, false, e);
         } else {
           log.warn(
               "Leader at: {} is not available. Index fetch failed by exception: {}",
-              leaderUrl,
+              leaderCoreUrl,
               errorMsg);
           return new IndexFetchResult(IndexFetchResult.FAILED_BY_EXCEPTION_MESSAGE, false, e);
         }
@@ -1985,14 +1993,14 @@ public class IndexFetcher {
 
       // TODO use shardhandler
       try (SolrClient client =
-          new Builder(leaderUrl)
+          new Builder(leaderBaseUrl)
               .withHttpClient(myHttpClient)
               .withResponseParser(null)
               .withConnectionTimeout(connTimeout, TimeUnit.MILLISECONDS)
               .withSocketTimeout(soTimeout, TimeUnit.MILLISECONDS)
               .build()) {
         QueryRequest req = new QueryRequest(params);
-        response = client.request(req);
+        response = client.request(req, leaderCoreName);
         is = (InputStream) response.get("stream");
         if (useInternalCompression) {
           is = new InflaterInputStream(is);
@@ -2118,13 +2126,13 @@ public class IndexFetcher {
 
     // TODO use shardhandler
     try (SolrClient client =
-        new HttpSolrClient.Builder(leaderUrl)
+        new HttpSolrClient.Builder(leaderBaseUrl)
             .withHttpClient(myHttpClient)
             .withConnectionTimeout(connTimeout, TimeUnit.MILLISECONDS)
             .withSocketTimeout(soTimeout, TimeUnit.MILLISECONDS)
             .build()) {
       QueryRequest request = new QueryRequest(params);
-      return client.request(request);
+      return client.request(request, leaderCoreName);
     }
   }
 
@@ -2133,8 +2141,8 @@ public class IndexFetcher {
     HttpClientUtil.close(myHttpClient);
   }
 
-  String getLeaderUrl() {
-    return leaderUrl;
+  String getLeaderCoreUrl() {
+    return leaderCoreUrl;
   }
 
   private static final int MAX_RETRIES = 5;
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index 6d674464ffc..a5451dc1b2b 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -905,7 +905,7 @@ public class ReplicationHandler extends RequestHandlerBase
             map -> {
               IndexFetcher fetcher = currentIndexFetcher;
               if (fetcher != null) {
-                map.put(LEADER_URL, fetcher.getLeaderUrl());
+                map.put(LEADER_URL, fetcher.getLeaderCoreUrl());
                 if (getPollInterval() != null) {
                   map.put(POLL_INTERVAL, getPollInterval());
                 }
@@ -987,7 +987,7 @@ public class ReplicationHandler extends RequestHandlerBase
           follower.add(ERR_STATUS, "invalid_leader");
         }
       }
-      follower.add(LEADER_URL, fetcher.getLeaderUrl());
+      follower.add(LEADER_URL, fetcher.getLeaderCoreUrl());
       if (getPollInterval() != null) {
         follower.add(POLL_INTERVAL, getPollInterval());
       }
diff --git a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
index 9302598777d..f2cf3dbeb85 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -845,8 +845,8 @@ public class SolrConfigHandler extends RequestHandlerBase
     // course)
     List<PerReplicaCallable> concurrentTasks = new ArrayList<>();
 
-    for (String coreUrl : getActiveReplicaCoreUrls(zkController, collection)) {
-      PerReplicaCallable e = new PerReplicaCallable(coreUrl, prop, expectedVersion, maxWaitSecs);
+    for (Replica replica : getActiveReplicas(zkController, collection)) {
+      PerReplicaCallable e = new PerReplicaCallable(replica, prop, expectedVersion, maxWaitSecs);
       concurrentTasks.add(e);
     }
     if (concurrentTasks.isEmpty()) return; // nothing to wait for ...
@@ -882,7 +882,7 @@ public class SolrConfigHandler extends RequestHandlerBase
         }
 
         if (!success) {
-          String coreUrl = concurrentTasks.get(f).coreUrl;
+          String coreUrl = concurrentTasks.get(f).replica.getCoreUrl();
           log.warn("Core {} could not get the expected version {}", coreUrl, expectedVersion);
           if (failedList == null) failedList = new ArrayList<>();
           failedList.add(coreUrl);
@@ -922,9 +922,8 @@ public class SolrConfigHandler extends RequestHandlerBase
     }
   }
 
-  public static List<String> getActiveReplicaCoreUrls(
-      ZkController zkController, String collection) {
-    List<String> activeReplicaCoreUrls = new ArrayList<>();
+  public static List<Replica> getActiveReplicas(ZkController zkController, String collection) {
+    List<Replica> activeReplicas = new ArrayList<>();
     ClusterState clusterState = zkController.getZkStateReader().getClusterState();
     Set<String> liveNodes = clusterState.getLiveNodes();
     final DocCollection docCollection = clusterState.getCollectionOrNull(collection);
@@ -939,13 +938,13 @@ public class SolrConfigHandler extends RequestHandlerBase
             Replica replica = entry.getValue();
             if (replica.getState() == Replica.State.ACTIVE
                 && liveNodes.contains(replica.getNodeName())) {
-              activeReplicaCoreUrls.add(replica.getCoreUrl());
+              activeReplicas.add(replica);
             }
           }
         }
       }
     }
-    return activeReplicaCoreUrls;
+    return activeReplicas;
   }
 
   @Override
@@ -962,15 +961,15 @@ public class SolrConfigHandler extends RequestHandlerBase
 
   private static class PerReplicaCallable extends DataStoreSolrRequest<SolrResponse>
       implements Callable<Boolean> {
-    String coreUrl;
+    Replica replica;
     String prop;
     int expectedZkVersion;
     Number remoteVersion = null;
     int maxWait;
 
-    PerReplicaCallable(String coreUrl, String prop, int expectedZkVersion, int maxWait) {
+    PerReplicaCallable(Replica replica, String prop, int expectedZkVersion, int maxWait) {
       super(METHOD.GET, "/config/" + ZNODEVER);
-      this.coreUrl = coreUrl;
+      this.replica = replica;
       this.expectedZkVersion = expectedZkVersion;
       this.prop = prop;
       this.maxWait = maxWait;
@@ -987,7 +986,10 @@ public class SolrConfigHandler extends RequestHandlerBase
     public Boolean call() throws Exception {
       final RTimer timer = new RTimer();
       int attempts = 0;
-      try (HttpSolrClient solr = new HttpSolrClient.Builder(coreUrl).build()) {
+      try (HttpSolrClient solr =
+          new HttpSolrClient.Builder(replica.getBaseUrl())
+              .withDefaultDataStore(replica.getCoreName())
+              .build()) {
         // eventually, this loop will get killed by the ExecutorService's timeout
         while (true) {
           try {
@@ -1012,13 +1014,13 @@ public class SolrConfigHandler extends RequestHandlerBase
               log.info(
                   formatString(
                       "Could not get expectedVersion {0} from {1} for prop {2}   after {3} attempts",
-                      expectedZkVersion, coreUrl, prop, attempts));
+                      expectedZkVersion, replica.getCoreUrl(), prop, attempts));
             }
           } catch (Exception e) {
             if (e instanceof InterruptedException) {
               break; // stop looping
             } else {
-              log.warn("Failed to get /schema/zkversion from {} due to: ", coreUrl, e);
+              log.warn("Failed to get /schema/zkversion from {} due to: ", replica.getCoreUrl(), e);
             }
           }
         }
diff --git a/solr/core/src/java/org/apache/solr/handler/component/IterativeMergeStrategy.java b/solr/core/src/java/org/apache/solr/handler/component/IterativeMergeStrategy.java
index bbd4ec9f42c..350d60174e8 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/IterativeMergeStrategy.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/IterativeMergeStrategy.java
@@ -35,6 +35,7 @@ import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.common.util.URLUtil;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -90,9 +91,14 @@ public abstract class IterativeMergeStrategy implements MergeStrategy {
     private ShardResponse originalShardResponse;
 
     public CallBack(ShardResponse originalShardResponse, QueryRequest req) {
-
+      final String shardBaseUrl = URLUtil.extractBaseUrl(originalShardResponse.getShardAddress());
+      final String shardCoreName =
+          URLUtil.extractCoreFromCoreUrl(originalShardResponse.getShardAddress());
       this.solrClient =
-          new Builder(originalShardResponse.getShardAddress()).withHttpClient(httpClient).build();
+          new Builder(shardBaseUrl)
+              .withDefaultDataStore(shardCoreName)
+              .withHttpClient(httpClient)
+              .build();
       this.req = req;
       this.originalShardResponse = originalShardResponse;
       req.setMethod(SolrRequest.METHOD.POST);
diff --git a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
index 2d6df7b8415..3ca6e7b8389 100644
--- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
@@ -54,7 +54,6 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
@@ -62,6 +61,7 @@ import org.apache.solr.common.util.CollectionUtil;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.common.util.URLUtil;
 import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrResourceLoader;
@@ -237,8 +237,10 @@ public final class ManagedIndexSchema extends IndexSchema {
     // get a list of active replica cores to query for the schema zk version (skipping this core of
     // course)
     List<GetZkSchemaVersionCallable> concurrentTasks = new ArrayList<>();
-    for (String coreUrl : getActiveReplicaCoreUrls(zkController, collection, localCoreNodeName))
-      concurrentTasks.add(new GetZkSchemaVersionCallable(coreUrl, schemaZkVersion, zkController));
+    for (Replica core : getActiveReplicas(zkController, collection, localCoreNodeName))
+      concurrentTasks.add(
+          new GetZkSchemaVersionCallable(
+              core.getBaseUrl(), core.getCoreName(), schemaZkVersion, zkController));
     if (concurrentTasks.isEmpty()) return; // nothing to wait for ...
 
     if (log.isInfoEnabled()) {
@@ -274,7 +276,8 @@ public final class ManagedIndexSchema extends IndexSchema {
         }
 
         if (vers == -1) {
-          String coreUrl = concurrentTasks.get(f).coreUrl;
+          final String coreUrl =
+              URLUtil.buildCoreUrl(concurrentTasks.get(f).baseUrl, concurrentTasks.get(f).coreName);
           log.warn(
               "Core {} version mismatch! Expected {} but got {}", coreUrl, schemaZkVersion, vers);
           if (failedList == null) failedList = new ArrayList<>();
@@ -318,9 +321,9 @@ public final class ManagedIndexSchema extends IndexSchema {
     }
   }
 
-  private static List<String> getActiveReplicaCoreUrls(
+  private static List<Replica> getActiveReplicas(
       ZkController zkController, String collection, String localCoreNodeName) {
-    List<String> activeReplicaCoreUrls = new ArrayList<>();
+    List<Replica> activeReplicas = new ArrayList<>();
     ZkStateReader zkStateReader = zkController.getZkStateReader();
     ClusterState clusterState = zkStateReader.getClusterState();
     Set<String> liveNodes = clusterState.getLiveNodes();
@@ -335,27 +338,29 @@ public final class ManagedIndexSchema extends IndexSchema {
             if (!localCoreNodeName.equals(replica.getName())
                 && replica.getState() == Replica.State.ACTIVE
                 && liveNodes.contains(replica.getNodeName())) {
-              ZkCoreNodeProps replicaCoreProps = new ZkCoreNodeProps(replica);
-              activeReplicaCoreUrls.add(replicaCoreProps.getCoreUrl());
+              activeReplicas.add(replica);
             }
           }
         }
       }
     }
-    return activeReplicaCoreUrls;
+    return activeReplicas;
   }
 
   private static class GetZkSchemaVersionCallable extends DataStoreSolrRequest<SolrResponse>
       implements Callable<Integer> {
 
     private final ZkController zkController;
-    private String coreUrl;
+    private String baseUrl;
+    private String coreName;
     private int expectedZkVersion;
 
-    GetZkSchemaVersionCallable(String coreUrl, int expectedZkVersion, ZkController zkController) {
+    GetZkSchemaVersionCallable(
+        String baseUrl, String coreName, int expectedZkVersion, ZkController zkController) {
       super(METHOD.GET, "/schema/zkversion");
       this.zkController = zkController;
-      this.coreUrl = coreUrl;
+      this.baseUrl = baseUrl;
+      this.coreName = coreName;
       this.expectedZkVersion = expectedZkVersion;
     }
 
@@ -369,7 +374,8 @@ public final class ManagedIndexSchema extends IndexSchema {
     @Override
     public Integer call() throws Exception {
       int remoteVersion = -1;
-      try (HttpSolrClient solr = new HttpSolrClient.Builder(coreUrl).build()) {
+      try (HttpSolrClient solr =
+          new HttpSolrClient.Builder(baseUrl).withDefaultDataStore(coreName).build()) {
         // eventually, this loop will get killed by the ExecutorService's timeout
         while (remoteVersion == -1
             || (remoteVersion < expectedZkVersion
@@ -385,7 +391,7 @@ public final class ManagedIndexSchema extends IndexSchema {
               // Thread.sleep(1000); // slight delay before requesting version again
               log.error(
                   "Replica {} returned schema version {} and has not applied schema version {}",
-                  coreUrl,
+                  coreName,
                   remoteVersion,
                   expectedZkVersion);
             }
@@ -395,7 +401,7 @@ public final class ManagedIndexSchema extends IndexSchema {
               Thread.currentThread().interrupt();
               break; // stop looping
             } else {
-              log.warn("Failed to get /schema/zkversion from {} due to: ", coreUrl, e);
+              log.warn("Failed to get /schema/zkversion from {} due to: ", baseUrl, e);
             }
           }
         }
diff --git a/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java b/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java
index d2ab56829e3..883f179b528 100644
--- a/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java
+++ b/solr/core/src/java/org/apache/solr/update/PeerSyncWithLeader.java
@@ -39,6 +39,7 @@ import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.URLUtil;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.metrics.SolrMetricProducer;
@@ -79,7 +80,14 @@ public class PeerSyncWithLeader implements SolrMetricProducer {
     this.uhandler = core.getUpdateHandler();
     this.ulog = uhandler.getUpdateLog();
     HttpClient httpClient = core.getCoreContainer().getUpdateShardHandler().getDefaultHttpClient();
-    this.clientToLeader = new HttpSolrClient.Builder(leaderUrl).withHttpClient(httpClient).build();
+
+    final var leaderBaseUrl = URLUtil.extractBaseUrl(leaderUrl);
+    final var coreName = URLUtil.extractCoreFromCoreUrl(leaderUrl);
+    this.clientToLeader =
+        new HttpSolrClient.Builder(leaderBaseUrl)
+            .withDefaultDataStore(coreName)
+            .withHttpClient(httpClient)
+            .build();
 
     this.updater = new PeerSync.Updater(msg(), core);
 
diff --git a/solr/core/src/java/org/apache/solr/update/StreamingSolrClients.java b/solr/core/src/java/org/apache/solr/update/StreamingSolrClients.java
index d441fc989f7..2bdb512d0fc 100644
--- a/solr/core/src/java/org/apache/solr/update/StreamingSolrClients.java
+++ b/solr/core/src/java/org/apache/solr/update/StreamingSolrClients.java
@@ -30,6 +30,7 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.impl.ConcurrentUpdateHttp2SolrClient;
 import org.apache.solr.client.solrj.impl.Http2SolrClient;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.update.SolrCmdDistributor.SolrError;
 import org.eclipse.jetty.client.api.Response;
 import org.slf4j.Logger;
@@ -70,8 +71,12 @@ public class StreamingSolrClients {
       // NOTE: increasing to more than 1 threadCount for the client could cause updates to be
       // reordered on a greater scale since the current behavior is to only increase the number of
       // connections/Runners when the queue is more than half full.
+      final var defaultCore =
+          StrUtils.isNotBlank(req.node.getCoreName()) ? req.node.getCoreName() : null;
       client =
-          new ErrorReportingConcurrentUpdateSolrClient.Builder(url, httpClient, req, errors)
+          new ErrorReportingConcurrentUpdateSolrClient.Builder(
+                  req.node.getBaseUrl(), httpClient, req, errors)
+              .withDefaultDataStore(defaultCore)
               .withQueueSize(100)
               .withThreadCount(runnerCount)
               .withExecutorService(updateExecutor)
diff --git a/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java b/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java
index c1a05d00987..4b7b0e07ed9 100644
--- a/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java
+++ b/solr/core/src/test/org/apache/solr/core/ConfigureRecoveryStrategyTest.java
@@ -18,6 +18,7 @@ package org.apache.solr.core;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.util.Set;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.cloud.RecoveryStrategy;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
@@ -67,10 +68,11 @@ public class ConfigureRecoveryStrategyTest extends SolrTestCaseJ4 {
   }
 
   public void testAlmostAllMethodsAreFinal() {
+    final var expectedNonFinals = Set.of("getReplicateLeaderUrl");
     for (Method m : RecoveryStrategy.class.getDeclaredMethods()) {
       if (Modifier.isStatic(m.getModifiers()) || Modifier.isPrivate(m.getModifiers())) continue;
       final String methodName = m.getName();
-      if ("getReplicateLeaderUrl".equals(methodName)) {
+      if (expectedNonFinals.contains(methodName)) {
         assertFalse(m.toString(), Modifier.isFinal(m.getModifiers()));
       } else {
         assertTrue(m.toString(), Modifier.isFinal(m.getModifiers()));
diff --git a/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java b/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
index 57866f8a716..f41874b9525 100644
--- a/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
@@ -171,7 +171,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
               ZkStateReader.BASE_URL_PROP,
               ((HttpSolrClient) controlClient).getBaseURL(),
               ZkStateReader.CORE_NAME_PROP,
-              "");
+              controlClient.getDefaultCollection());
       nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
 
       // add one doc to controlClient
@@ -195,7 +195,10 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
       client = (HttpSolrClient) clients.get(0);
       nodeProps =
           new ZkNodeProps(
-              ZkStateReader.BASE_URL_PROP, client.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
+              ZkStateReader.BASE_URL_PROP,
+              client.getBaseURL(),
+              ZkStateReader.CORE_NAME_PROP,
+              client.getDefaultCollection());
       nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
     }
     int id2;
@@ -286,7 +289,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
                   ZkStateReader.BASE_URL_PROP,
                   httpClient.getBaseURL(),
                   ZkStateReader.CORE_NAME_PROP,
-                  "");
+                  httpClient.getDefaultCollection());
           nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
         }
         AddUpdateCommand c = new AddUpdateCommand(null);
@@ -306,7 +309,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
                 ZkStateReader.BASE_URL_PROP,
                 httpClient.getBaseURL(),
                 ZkStateReader.CORE_NAME_PROP,
-                "");
+                httpClient.getDefaultCollection());
 
         nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
       }
@@ -393,7 +396,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
               ZkStateReader.BASE_URL_PROP,
               solrclient.getBaseURL(),
               ZkStateReader.CORE_NAME_PROP,
-              "");
+              solrclient.getDefaultCollection());
       Node retryNode =
           new StdNode(new ZkCoreNodeProps(nodeProps), "collection1", "shard1", 5) {
             @Override
@@ -457,7 +460,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
               ZkStateReader.BASE_URL_PROP,
               solrclient.getBaseURL(),
               ZkStateReader.CORE_NAME_PROP,
-              "");
+              solrclient.getDefaultCollection());
       if (nodeType == NodeType.FORWARD) {
         nodes.add(
             new ForwardNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1", 5) {
@@ -514,7 +517,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
               ZkStateReader.BASE_URL_PROP,
               solrClient1.getBaseURL(),
               ZkStateReader.CORE_NAME_PROP,
-              "");
+              solrClient1.getDefaultCollection());
       Node retryNode;
       if (nodeType == NodeType.FORWARD) {
         retryNode =
@@ -638,7 +641,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
               ZkStateReader.BASE_URL_PROP,
               solrclient.getBaseURL(),
               ZkStateReader.CORE_NAME_PROP,
-              "");
+              solrclient.getDefaultCollection());
       Node retryNode;
       if (nodeType == NodeType.FORWARD) {
         retryNode =
@@ -697,7 +700,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
               ZkStateReader.BASE_URL_PROP,
               solrclient.getBaseURL(),
               ZkStateReader.CORE_NAME_PROP,
-              "");
+              solrclient.getDefaultCollection());
 
       final AtomicInteger retries = new AtomicInteger();
       Node retryNode;
@@ -760,7 +763,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
               ZkStateReader.BASE_URL_PROP,
               solrclient.getBaseURL(),
               ZkStateReader.CORE_NAME_PROP,
-              "");
+              solrclient.getDefaultCollection());
       ForwardNode retryNode =
           new ForwardNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1", 5) {
             @Override
@@ -808,7 +811,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
               ZkStateReader.BASE_URL_PROP,
               solrclient.getBaseURL(),
               ZkStateReader.CORE_NAME_PROP,
-              "");
+              solrclient.getDefaultCollection());
       Node retryNode =
           new StdNode(new ZkCoreNodeProps(nodeProps), "collection1", "shard1", 5) {
             @Override
@@ -852,7 +855,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
                       ZkStateReader.BASE_URL_PROP,
                       solrclient.getBaseURL(),
                       ZkStateReader.CORE_NAME_PROP,
-                      "");
+                      solrclient.getDefaultCollection());
               this.nodeProps = new ZkCoreNodeProps(leaderProps);
 
               return super.checkRetry(err);
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java
index 375b3c38742..74a225d8a90 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseHttpClusterStateProvider.java
@@ -80,6 +80,7 @@ public abstract class BaseHttpClusterStateProvider implements ClusterStateProvid
     }
   }
 
+  /** Create a SolrClient implementation that uses the specified Solr node URL */
   protected abstract SolrClient getSolrClient(String baseUrl);
 
   @Override
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
index 01a8c7377ca..de42b4be2b1 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
@@ -297,7 +297,7 @@ public class HttpSolrClient extends BaseHttpSolrClient {
       final SolrRequest<?> request, final ResponseParser processor)
       throws SolrServerException, IOException {
     HttpUriRequestResponse mrr = new HttpUriRequestResponse();
-    final HttpRequestBase method = createMethod(request, null);
+    final HttpRequestBase method = createMethod(request, defaultCollection);
     ExecutorService pool =
         ExecutorUtil.newMDCAwareFixedThreadPool(1, new SolrNamedThreadFactory("httpUriRequest"));
     try {
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/URLUtil.java b/solr/solrj/src/java/org/apache/solr/common/util/URLUtil.java
index 2f5b11b6f18..026c9a959a6 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/URLUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/URLUtil.java
@@ -45,4 +45,41 @@ public class URLUtil {
 
     return null;
   }
+
+  /**
+   * @param coreUrl a URL pointing to a specific "core" or collection (i.e. that adheres loosely to
+   *     the form "scheme://host:port/solr/coreName")
+   * @return a URL pointing to the Solr node's root path
+   */
+  public static String extractBaseUrl(String coreUrl) {
+    coreUrl = removeTrailingSlashIfPresent(coreUrl);
+
+    // Remove the core name and return
+    return coreUrl.substring(0, coreUrl.lastIndexOf("/"));
+  }
+
+  public static String extractCoreFromCoreUrl(String coreUrl) {
+    coreUrl = removeTrailingSlashIfPresent(coreUrl);
+
+    return coreUrl.substring(coreUrl.lastIndexOf("/") + 1);
+  }
+
+  /**
+   * Create a core URL (e.g. "http://localhost:8983/solr/myCore") from its individual components
+   *
+   * @param baseUrl a Solr "base URL" (e.g. "http://localhost:8983/solr/")
+   * @param coreName the name of a Solr core or collection (with no leading or trailing slashes)
+   */
+  public static String buildCoreUrl(String baseUrl, String coreName) {
+    baseUrl = removeTrailingSlashIfPresent(baseUrl);
+    return baseUrl + "/" + coreName;
+  }
+
+  private static String removeTrailingSlashIfPresent(String url) {
+    if (url.endsWith("/")) {
+      return url.substring(0, url.length() - 1);
+    }
+
+    return url;
+  }
 }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestSuggesterResponse.java b/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestSuggesterResponse.java
index 3f907622a58..b595b23fd61 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestSuggesterResponse.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestSuggesterResponse.java
@@ -143,7 +143,8 @@ public class TestSuggesterResponse extends SolrJettyTestBase {
     final ResponseParser randomParser =
         random().nextBoolean() ? new BinaryResponseParser() : new XMLResponseParser();
     return new HttpSolrClient.Builder()
-        .withBaseSolrUrl(getCoreUrl())
+        .withBaseSolrUrl(getBaseUrl())
+        .withDefaultDataStore(DEFAULT_TEST_CORENAME)
         .withResponseParser(randomParser)
         .build();
   }
diff --git a/solr/solrj/src/test/org/apache/solr/common/util/URLUtilTest.java b/solr/solrj/src/test/org/apache/solr/common/util/URLUtilTest.java
index c7560a4b60d..0394d71d46d 100644
--- a/solr/solrj/src/test/org/apache/solr/common/util/URLUtilTest.java
+++ b/solr/solrj/src/test/org/apache/solr/common/util/URLUtilTest.java
@@ -32,4 +32,67 @@ public class URLUtilTest extends SolrTestCase {
     assertEquals("http://", URLUtil.getScheme("http://host:1928"));
     assertEquals("https://", URLUtil.getScheme("https://host:1928"));
   }
+
+  @Test
+  public void testCanExtractBaseUrl() {
+    assertEquals(
+        "http://localhost:8983/solr",
+        URLUtil.extractBaseUrl("http://localhost:8983/solr/techproducts"));
+    assertEquals(
+        "http://localhost:8983/solr",
+        URLUtil.extractBaseUrl("http://localhost:8983/solr/techproducts/"));
+
+    assertEquals(
+        "http://localhost/solr", URLUtil.extractBaseUrl("http://localhost/solr/techproducts"));
+    assertEquals(
+        "http://localhost/solr", URLUtil.extractBaseUrl("http://localhost/solr/techproducts/"));
+
+    assertEquals(
+        "http://localhost:8983/root/solr",
+        URLUtil.extractBaseUrl("http://localhost:8983/root/solr/techproducts"));
+    assertEquals(
+        "http://localhost:8983/root/solr",
+        URLUtil.extractBaseUrl("http://localhost:8983/root/solr/techproducts/"));
+  }
+
+  @Test
+  public void testCanExtractCoreNameFromCoreUrl() {
+    assertEquals(
+        "techproducts", URLUtil.extractCoreFromCoreUrl("http://localhost:8983/solr/techproducts"));
+    assertEquals(
+        "techproducts", URLUtil.extractCoreFromCoreUrl("http://localhost:8983/solr/techproducts/"));
+
+    assertEquals(
+        "techproducts", URLUtil.extractCoreFromCoreUrl("http://localhost/solr/techproducts"));
+    assertEquals(
+        "techproducts", URLUtil.extractCoreFromCoreUrl("http://localhost/solr/techproducts/"));
+
+    assertEquals(
+        "techproducts",
+        URLUtil.extractCoreFromCoreUrl("http://localhost:8983/root/solr/techproducts"));
+    assertEquals(
+        "techproducts",
+        URLUtil.extractCoreFromCoreUrl("http://localhost:8983/root/solr/techproducts/"));
+
+    // Exercises most of the edge cases that SolrIdentifierValidator allows
+    assertEquals(
+        "sTrAnGe-name.for_core",
+        URLUtil.extractCoreFromCoreUrl("http://localhost:8983/solr/sTrAnGe-name.for_core"));
+    assertEquals(
+        "sTrAnGe-name.for_core",
+        URLUtil.extractCoreFromCoreUrl("http://localhost:8983/solr/sTrAnGe-name.for_core/"));
+  }
+
+  @Test
+  public void testCanBuildCoreUrl() {
+    assertEquals(
+        "http://localhost:8983/solr/techproducts",
+        URLUtil.buildCoreUrl("http://localhost:8983/solr", "techproducts"));
+    assertEquals(
+        "http://localhost:8983/solr/techproducts",
+        URLUtil.buildCoreUrl("http://localhost:8983/solr/", "techproducts"));
+    assertEquals(
+        "http://localhost:8983/solr/sTrAnGe-name.for_core",
+        URLUtil.buildCoreUrl("http://localhost:8983/solr", "sTrAnGe-name.for_core"));
+  }
 }
diff --git a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
index d466d993a48..0c810dbcfdf 100644
--- a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
@@ -454,16 +454,7 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
   }
 
   protected SolrClient createNewSolrClient(int port) {
-    return getHttpSolrClient(getServerUrl(port));
-  }
-
-  protected String getServerUrl(int port) {
-    String baseUrl = buildUrl(port);
-    if (baseUrl.endsWith("/")) {
-      return baseUrl + DEFAULT_TEST_CORENAME;
-    } else {
-      return baseUrl + "/" + DEFAULT_TEST_CORENAME;
-    }
+    return getHttpSolrClient(buildUrl(port), DEFAULT_TEST_CORENAME);
   }
 
   protected static void addFields(SolrInputDocument doc, Object... fields) {
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 0099320f875..471886d5963 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -2660,6 +2660,14 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
     return new HttpSolrClient.Builder(url).build();
   }
 
+  /**
+   * This method creates a basic HttpSolrClient. Tests that want to control the creation process
+   * should use the {@link org.apache.solr.client.solrj.impl.Http2SolrClient.Builder} class directly
+   */
+  public static HttpSolrClient getHttpSolrClient(String url, String defaultCoreName) {
+    return new HttpSolrClient.Builder(url).withDefaultDataStore(defaultCoreName).build();
+  }
+
   /**
    * Returns a randomly generated Date in the appropriate Solr external (input) format
    *