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/09 13:48:24 UTC

(solr) branch main updated: SOLR-17066: Ensure default only applied to coll-aware requests (#2177)

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 1043632f56c SOLR-17066: Ensure default only applied to coll-aware requests (#2177)
1043632f56c is described below

commit 1043632f56c4f18c26ede20bc0657491383fc71f
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Tue Jan 9 08:48:17 2024 -0500

    SOLR-17066: Ensure default only applied to coll-aware requests (#2177)
    
    SolrClients set up with a default data store still have issues making
    admin requests, since the SolrClient path-building logic doesn't
    differentiate between admin and non-admin requests.
    
    This commit fixes this by adding a boolean SolrRequest method,
    `requiresDataStore()`, which allows SolrClients to only selectively
    use the defaultCollection for requests that are supposed to be
    use a core/collection.
    
    The default-collection builder method "withDefaultCollection" has
    also been renamed to the more general "withDefaultDataStore" to
    match the naming used in the SolrRequest method.
---
 solr/CHANGES.txt                                   |  5 +++--
 .../org/apache/solr/handler/SolrConfigHandler.java |  3 ++-
 .../org/apache/solr/schema/ManagedIndexSchema.java |  3 ++-
 .../org/apache/solr/client/solrj/SolrRequest.java  | 13 ++++++++++++
 .../client/solrj/impl/CloudHttp2SolrClient.java    |  4 ++--
 .../client/solrj/impl/CloudLegacySolrClient.java   | 12 ++++++++++-
 .../impl/ConcurrentUpdateHttp2SolrClient.java      | 14 +++++++------
 .../solrj/impl/ConcurrentUpdateSolrClient.java     |  5 +++--
 .../solr/client/solrj/impl/Http2SolrClient.java    | 13 ++++++------
 .../solr/client/solrj/impl/HttpSolrClient.java     |  6 ++++--
 .../solr/client/solrj/impl/LBHttp2SolrClient.java  | 10 +++++-----
 .../solr/client/solrj/impl/LBHttpSolrClient.java   |  4 ++--
 .../solr/client/solrj/impl/LBSolrClient.java       |  4 +++-
 .../solr/client/solrj/impl/SolrClientBuilder.java  |  7 ++++---
 .../solrj/request/AbstractUpdateRequest.java       |  3 +--
 ...chemaRequest.java => DataStoreSolrRequest.java} | 23 +++++-----------------
 .../client/solrj/request/DirectXmlRequest.java     |  4 ++--
 .../solrj/request/DocumentAnalysisRequest.java     |  3 +--
 .../client/solrj/request/FieldAnalysisRequest.java |  3 +--
 .../solr/client/solrj/request/LukeRequest.java     |  3 +--
 .../solr/client/solrj/request/QueryRequest.java    |  3 +--
 .../apache/solr/client/solrj/request/SolrPing.java |  3 +--
 .../request/schema/AbstractSchemaRequest.java      |  5 +++--
 .../apache/solr/client/solrj/util/ClientUtils.java | 14 +++++++++++++
 .../ConcurrentUpdateSolrClientBuilderTest.java     |  2 +-
 .../solrj/impl/HttpSolrClientBuilderTest.java      |  2 +-
 .../solrj/impl/LBHttpSolrClientBuilderTest.java    |  2 +-
 .../solr/client/solrj/util/ClientUtilsTest.java    | 19 ++++++++++++++++++
 .../java/org/apache/solr/cloud/ConfigRequest.java  |  3 ++-
 29 files changed, 123 insertions(+), 72 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0e0c6e30be1..d97f7dcd346 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -125,8 +125,9 @@ Improvements
 * SOLR-17063: Do not retain log param references in LogWatcher (Michael Gibney)
 
 * SOLR-17066: SolrClient builders now allow users to specify a "default" collection or core
-  using the `withDefaultCollection` method.  This is preferable to including the collection
-  in the base URL accepted by certain client implementations. (Jason Gerlowski)
+  using the `withDefaultDataStore` method  ("Cloud" client builders retain the more specific
+  `withDefaultCollection`).  Use of the Builder methods is preferable to including the
+  collection in the base URL accepted by certain client implementations. (Jason Gerlowski)
 
 * SOLR-15960: Unified use of system properties and environment variables (janhoy)
 
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 763ed3c11ab..9302598777d 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -59,6 +59,7 @@ import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.request.DataStoreSolrRequest;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.MapSerializable;
@@ -959,7 +960,7 @@ public class SolrConfigHandler extends RequestHandlerBase
     }
   }
 
-  private static class PerReplicaCallable extends SolrRequest<SolrResponse>
+  private static class PerReplicaCallable extends DataStoreSolrRequest<SolrResponse>
       implements Callable<Boolean> {
     String coreUrl;
     String prop;
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 b41728e1264..2d6df7b8415 100644
--- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
@@ -44,6 +44,7 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.DataStoreSolrRequest;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
@@ -344,7 +345,7 @@ public final class ManagedIndexSchema extends IndexSchema {
     return activeReplicaCoreUrls;
   }
 
-  private static class GetZkSchemaVersionCallable extends SolrRequest<SolrResponse>
+  private static class GetZkSchemaVersionCallable extends DataStoreSolrRequest<SolrResponse>
       implements Callable<Integer> {
 
     private final ZkController zkController;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
index d3f813f8bcb..acc0aae4130 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
@@ -175,6 +175,19 @@ public abstract class SolrRequest<T extends SolrResponse> implements Serializabl
 
   public abstract SolrParams getParams();
 
+  /**
+   * Determines whether this request should use or ignore any specified data stores (esp. {@link
+   * SolrClient#defaultCollection})
+   *
+   * <p>Many Solr requests target a particular data store (i.e. core or collection). But not all of
+   * them - many Solr APIs (e.g. security or other admin APIs) are agnostic of data stores entirely.
+   * This method gives these requests a way to opt out of using {@link SolrClient#defaultCollection}
+   * or other specified data stores.
+   */
+  public boolean requiresDataStore() {
+    return false;
+  }
+
   /**
    * @deprecated Please use {@link SolrRequest#getContentWriter(String)} instead.
    */
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 cc3861cc5b5..e00ed50c9e7 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
@@ -322,8 +322,8 @@ public class CloudHttp2SolrClient extends CloudSolrClient {
     }
 
     /** Sets the default collection for request. */
-    public Builder withDefaultCollection(String collection) {
-      this.defaultCollection = collection;
+    public Builder withDefaultCollection(String defaultCollection) {
+      this.defaultCollection = defaultCollection;
       return this;
     }
 
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 e1f3840cfbd..e3c493f1a61 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
@@ -64,7 +64,7 @@ public class CloudLegacySolrClient extends CloudSolrClient {
     super(builder.shardLeadersOnly, builder.parallelUpdates, builder.directUpdatesToLeadersOnly);
     this.stateProvider = builder.stateProvider;
     this.retryExpiryTimeNano = builder.retryExpiryTimeNano;
-    this.defaultCollection = builder.defaultCollection;
+    this.defaultCollection = builder.defaultDataStore;
     this.collectionStateCache.timeToLiveMs =
         TimeUnit.MILLISECONDS.convert(builder.timeToLiveSeconds, TimeUnit.SECONDS);
     this.clientIsInternal = builder.httpClient == null;
@@ -238,6 +238,16 @@ public class CloudLegacySolrClient extends CloudSolrClient {
       return this;
     }
 
+    /**
+     * Sets a default collection for all collection-based requests.
+     *
+     * <p>Identical to {@link #withDefaultDataStore(String)} for this builder
+     */
+    public Builder withDefaultCollection(String defaultCollection) {
+      this.defaultDataStore = defaultCollection;
+      return this;
+    }
+
     /** Provides a {@link HttpClient} for the builder to use when creating clients. */
     public Builder withLBHttpSolrClientBuilder(LBHttpSolrClient.Builder lbHttpSolrClientBuilder) {
       this.lbClientBuilder = lbHttpSolrClientBuilder;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClient.java
index 617e853db52..2ad0e7d9ae8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClient.java
@@ -34,6 +34,7 @@ import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient.Update;
 import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.UpdateParams;
@@ -149,7 +150,7 @@ public class ConcurrentUpdateHttp2SolrClient extends SolrClient {
     this.runners = new ArrayDeque<>();
     this.streamDeletes = builder.streamDeletes;
     this.basePath = builder.baseSolrUrl;
-    this.defaultCollection = builder.defaultCollection;
+    this.defaultCollection = builder.defaultDataStore;
     this.pollQueueTimeMillis = builder.pollQueueTimeMillis;
     this.stallTimeMillis = Integer.getInteger("solr.cloud.client.stallTime", 15000);
 
@@ -360,7 +361,8 @@ public class ConcurrentUpdateHttp2SolrClient extends SolrClient {
   @Override
   public NamedList<Object> request(final SolrRequest<?> request, String collection)
       throws SolrServerException, IOException {
-    if (collection == null) collection = defaultCollection;
+    if (ClientUtils.shouldApplyDefaultDataStore(collection, request))
+      collection = defaultCollection;
     if (!(request instanceof UpdateRequest)) {
       request.setBasePath(basePath);
       return client.request(request, collection);
@@ -699,7 +701,7 @@ public class ConcurrentUpdateHttp2SolrClient extends SolrClient {
   public static class Builder {
     protected Http2SolrClient client;
     protected String baseSolrUrl;
-    protected String defaultCollection;
+    protected String defaultDataStore;
     protected int queueSize = 10;
     protected int threadCount;
     protected ExecutorService executorService;
@@ -790,9 +792,9 @@ public class ConcurrentUpdateHttp2SolrClient extends SolrClient {
       return this;
     }
 
-    /** Sets a default collection for collection-based requests. */
-    public Builder withDefaultCollection(String defaultCollection) {
-      this.defaultCollection = defaultCollection;
+    /** Sets a default data store for core- or collection-based requests. */
+    public Builder withDefaultDataStore(String defaultCoreOrCollection) {
+      this.defaultDataStore = defaultCoreOrCollection;
       return this;
     }
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
index 6bac5c0457b..e882c24b119 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
@@ -116,7 +116,7 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
     this.soTimeout = builder.socketTimeoutMillis;
     this.pollQueueTimeMillis = builder.pollQueueTime;
     this.stallTimeMillis = Integer.getInteger("solr.cloud.client.stallTime", 15000);
-    this.defaultCollection = builder.defaultCollection;
+    this.defaultCollection = builder.defaultDataStore;
 
     // make sure the stall time is larger than the polling time
     // to give a chance for the queue to change
@@ -476,7 +476,8 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
   @Override
   public NamedList<Object> request(final SolrRequest<?> request, String collection)
       throws SolrServerException, IOException {
-    if (collection == null) collection = defaultCollection;
+    if (ClientUtils.shouldApplyDefaultDataStore(collection, request))
+      collection = defaultCollection;
     if (!(request instanceof UpdateRequest)) {
       return client.request(request, collection);
     }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
index af5164f2611..ed618137e1c 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
@@ -191,7 +191,7 @@ public class Http2SolrClient extends SolrClient {
       this.parser = builder.responseParser;
     }
     updateDefaultMimeTypeForParser();
-    this.defaultCollection = builder.defaultCollection;
+    this.defaultCollection = builder.defaultDataStore;
     if (builder.requestTimeoutMillis != null) {
       this.requestTimeoutMillis = builder.requestTimeoutMillis;
     } else {
@@ -555,7 +555,8 @@ public class Http2SolrClient extends SolrClient {
   @Override
   public NamedList<Object> request(SolrRequest<?> solrRequest, String collection)
       throws SolrServerException, IOException {
-    if (collection == null) collection = defaultCollection;
+    if (ClientUtils.shouldApplyDefaultDataStore(collection, solrRequest))
+      collection = defaultCollection;
     String url = getRequestPath(solrRequest, collection);
     Throwable abortCause = null;
     Request req = null;
@@ -1071,7 +1072,7 @@ public class Http2SolrClient extends SolrClient {
     private ExecutorService executor;
     protected RequestWriter requestWriter;
     protected ResponseParser responseParser;
-    protected String defaultCollection;
+    protected String defaultDataStore;
     private Set<String> urlParamNames;
     private CookieStore cookieStore = getDefaultCookieStore();
     private String proxyHost;
@@ -1196,9 +1197,9 @@ public class Http2SolrClient extends SolrClient {
       return this;
     }
 
-    /** Sets a default collection for collection-based requests. */
-    public Builder withDefaultCollection(String defaultCollection) {
-      this.defaultCollection = defaultCollection;
+    /** Sets a default data store for core- or collection-based requests. */
+    public Builder withDefaultDataStore(String defaultCoreOrCollection) {
+      this.defaultDataStore = defaultCoreOrCollection;
       return this;
     }
 
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 9d11dbe7768..01a8c7377ca 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
@@ -78,6 +78,7 @@ import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.request.V2Request;
+import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -188,7 +189,7 @@ public class HttpSolrClient extends BaseHttpSolrClient {
     this.soTimeout = builder.socketTimeoutMillis;
     this.useMultiPartPost = builder.useMultiPartPost;
     this.urlParamNames = builder.urlParamNames;
-    this.defaultCollection = builder.defaultCollection;
+    this.defaultCollection = builder.defaultDataStore;
   }
 
   public Set<String> getUrlParamNames() {
@@ -243,7 +244,8 @@ public class HttpSolrClient extends BaseHttpSolrClient {
   public NamedList<Object> request(
       final SolrRequest<?> request, final ResponseParser processor, String collection)
       throws SolrServerException, IOException {
-    if (collection == null) collection = defaultCollection;
+    if (ClientUtils.shouldApplyDefaultDataStore(collection, request))
+      collection = defaultCollection;
     HttpRequestBase method = createMethod(request, collection);
     setBasicAuthHeader(request, method);
     if (request.getHeaders() != null) {
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttp2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttp2SolrClient.java
index 82650e525b1..3463776a8bc 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttp2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttp2SolrClient.java
@@ -89,7 +89,7 @@ public class LBHttp2SolrClient extends LBSolrClient {
     super(Arrays.asList(builder.baseSolrUrls));
     this.solrClient = builder.http2SolrClient;
     this.aliveCheckIntervalMillis = builder.aliveCheckIntervalMillis;
-    this.defaultCollection = builder.defaultCollection;
+    this.defaultCollection = builder.defaultDataStore;
   }
 
   @Override
@@ -259,7 +259,7 @@ public class LBHttp2SolrClient extends LBSolrClient {
     private final String[] baseSolrUrls;
     private long aliveCheckIntervalMillis =
         TimeUnit.MILLISECONDS.convert(60, TimeUnit.SECONDS); // 1 minute between checks
-    protected String defaultCollection;
+    protected String defaultDataStore;
 
     public Builder(Http2SolrClient http2Client, String... baseSolrUrls) {
       this.http2SolrClient = http2Client;
@@ -281,9 +281,9 @@ public class LBHttp2SolrClient extends LBSolrClient {
       return this;
     }
 
-    /** Sets a default collection for collection-based requests. */
-    public LBHttp2SolrClient.Builder withDefaultCollection(String defaultCollection) {
-      this.defaultCollection = defaultCollection;
+    /** Sets a default data store for core- or collection-based requests. */
+    public LBHttp2SolrClient.Builder withDefaultDataStore(String defaultCoreOrCollection) {
+      this.defaultDataStore = defaultCoreOrCollection;
       return this;
     }
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
index 923d651afb3..4a9a12cc332 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
@@ -93,9 +93,9 @@ public class LBHttpSolrClient extends LBSolrClient {
         builder.httpClient == null
             ? constructClient(builder.baseSolrUrls.toArray(new String[0]))
             : builder.httpClient;
-    this.defaultCollection = builder.defaultCollection;
+    this.defaultCollection = builder.defaultDataStore;
     if (httpSolrClientBuilder != null && this.defaultCollection != null) {
-      httpSolrClientBuilder.defaultCollection = this.defaultCollection;
+      httpSolrClientBuilder.defaultDataStore = this.defaultCollection;
     }
     this.connectionTimeoutMillis = builder.connectionTimeoutMillis;
     this.soTimeoutMillis = builder.socketTimeoutMillis;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBSolrClient.java
index 6acf04aea57..8824cc4873b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBSolrClient.java
@@ -49,6 +49,7 @@ import org.apache.solr.client.solrj.request.IsUpdateRequest;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
@@ -574,7 +575,8 @@ public abstract class LBSolrClient extends SolrClient {
     final int maxTries = (numServersToTry == null ? serverList.length : numServersToTry.intValue());
     int numServersTried = 0;
     Map<String, ServerWrapper> justFailed = null;
-    if (collection == null) collection = defaultCollection;
+    if (ClientUtils.shouldApplyDefaultDataStore(collection, request))
+      collection = defaultCollection;
 
     boolean timeAllowedExceeded = false;
     long timeAllowedNano = getTimeAllowedInNanos(request);
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientBuilder.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientBuilder.java
index aee3bf55f23..c08fa9ed734 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientBuilder.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientBuilder.java
@@ -41,7 +41,7 @@ public abstract class SolrClientBuilder<B extends SolrClientBuilder<B>> {
   protected int socketTimeoutMillis = 120000; // 120 seconds
   private boolean socketTimeoutMillisUpdate = false;
   protected boolean followRedirects = false;
-  protected String defaultCollection;
+  protected String defaultDataStore;
   protected Set<String> urlParamNames;
 
   /** The solution for the unchecked cast warning. */
@@ -98,8 +98,9 @@ public abstract class SolrClientBuilder<B extends SolrClientBuilder<B>> {
     return getThis();
   }
 
-  public B withDefaultCollection(String defaultCollection) {
-    this.defaultCollection = defaultCollection;
+  /** Sets a default data store for core- or collection-based requests. */
+  public B withDefaultDataStore(String defaultCoreOrCollection) {
+    this.defaultDataStore = defaultCoreOrCollection;
     return getThis();
   }
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/AbstractUpdateRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/AbstractUpdateRequest.java
index 0132954c81f..bbdba067ea4 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/AbstractUpdateRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/AbstractUpdateRequest.java
@@ -17,13 +17,12 @@
 package org.apache.solr.client.solrj.request;
 
 import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.UpdateParams;
 
 /** */
-public abstract class AbstractUpdateRequest extends SolrRequest<UpdateResponse>
+public abstract class AbstractUpdateRequest extends DataStoreSolrRequest<UpdateResponse>
     implements IsUpdateRequest {
   protected ModifiableSolrParams params;
   protected int commitWithin = -1;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/schema/AbstractSchemaRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/DataStoreSolrRequest.java
similarity index 62%
copy from solr/solrj/src/java/org/apache/solr/client/solrj/request/schema/AbstractSchemaRequest.java
copy to solr/solrj/src/java/org/apache/solr/client/solrj/request/DataStoreSolrRequest.java
index ed0f315b959..2972e99913b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/schema/AbstractSchemaRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/DataStoreSolrRequest.java
@@ -14,31 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.solr.client.solrj.request.schema;
+package org.apache.solr.client.solrj.request;
 
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
-import org.apache.solr.common.params.SolrParams;
 
-public abstract class AbstractSchemaRequest<T extends SolrResponse> extends SolrRequest<T> {
-  private SolrParams params = null;
-
-  public AbstractSchemaRequest(METHOD m, String path) {
+public abstract class DataStoreSolrRequest<T extends SolrResponse> extends SolrRequest<T> {
+  public DataStoreSolrRequest(METHOD m, String path) {
     super(m, path);
   }
 
-  public AbstractSchemaRequest(METHOD m, String path, SolrParams params) {
-    this(m, path);
-    this.params = params;
-  }
-
-  @Override
-  public SolrParams getParams() {
-    return params;
-  }
-
   @Override
-  public String getRequestType() {
-    return SolrRequestType.ADMIN.toString();
+  public boolean requiresDataStore() {
+    return true;
   }
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/DirectXmlRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/DirectXmlRequest.java
index 6b23853214e..8b2c09dbdd5 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/DirectXmlRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/DirectXmlRequest.java
@@ -17,7 +17,6 @@
 package org.apache.solr.client.solrj.request;
 
 import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.request.RequestWriter.StringPayloadContentWriter;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.client.solrj.util.ClientUtils;
@@ -28,7 +27,8 @@ import org.apache.solr.common.params.SolrParams;
  *
  * @since solr 1.3
  */
-public class DirectXmlRequest extends SolrRequest<UpdateResponse> implements IsUpdateRequest {
+public class DirectXmlRequest extends DataStoreSolrRequest<UpdateResponse>
+    implements IsUpdateRequest {
 
   final String xml;
   private SolrParams params;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/DocumentAnalysisRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/DocumentAnalysisRequest.java
index 78eb828f5b8..494b1e518ee 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/DocumentAnalysisRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/DocumentAnalysisRequest.java
@@ -26,7 +26,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.response.DocumentAnalysisResponse;
 import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.common.SolrInputDocument;
@@ -38,7 +37,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
  *
  * @since solr 1.4
  */
-public class DocumentAnalysisRequest extends SolrRequest<DocumentAnalysisResponse> {
+public class DocumentAnalysisRequest extends DataStoreSolrRequest<DocumentAnalysisResponse> {
 
   private List<SolrInputDocument> documents = new ArrayList<>();
   private String query;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/FieldAnalysisRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/FieldAnalysisRequest.java
index 2dcfbca7370..5458ac99dde 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/FieldAnalysisRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/FieldAnalysisRequest.java
@@ -19,7 +19,6 @@ package org.apache.solr.client.solrj.request;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.response.FieldAnalysisResponse;
 import org.apache.solr.common.params.AnalysisParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -30,7 +29,7 @@ import org.apache.solr.common.params.SolrParams;
  *
  * @since solr.14
  */
-public class FieldAnalysisRequest extends SolrRequest<FieldAnalysisResponse> {
+public class FieldAnalysisRequest extends DataStoreSolrRequest<FieldAnalysisResponse> {
 
   private String fieldValue;
   private String query;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/LukeRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/LukeRequest.java
index 3e28a258faa..a3b3a9d265b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/LukeRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/LukeRequest.java
@@ -19,7 +19,6 @@ package org.apache.solr.client.solrj.request;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.response.LukeResponse;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -28,7 +27,7 @@ import org.apache.solr.common.params.SolrParams;
 /**
  * @since solr 1.3
  */
-public class LukeRequest extends SolrRequest<LukeResponse> {
+public class LukeRequest extends DataStoreSolrRequest<LukeResponse> {
 
   private List<String> fields;
   private int numTerms = -1;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/QueryRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/QueryRequest.java
index bb839967288..cb01201842d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/QueryRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/QueryRequest.java
@@ -17,7 +17,6 @@
 package org.apache.solr.client.solrj.request;
 
 import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
@@ -25,7 +24,7 @@ import org.apache.solr.common.params.SolrParams;
 /**
  * @since solr 1.3
  */
-public class QueryRequest extends SolrRequest<QueryResponse> {
+public class QueryRequest extends DataStoreSolrRequest<QueryResponse> {
 
   private SolrParams query;
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/SolrPing.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/SolrPing.java
index 342f68bc40b..ee1e89cbb89 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/SolrPing.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/SolrPing.java
@@ -17,7 +17,6 @@
 package org.apache.solr.client.solrj.request;
 
 import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.response.SolrPingResponse;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -29,7 +28,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
  *
  * @since solr 1.3
  */
-public class SolrPing extends SolrRequest<SolrPingResponse> {
+public class SolrPing extends DataStoreSolrRequest<SolrPingResponse> {
 
   /** serialVersionUID. */
   private static final long serialVersionUID = 5828246236669090017L;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/schema/AbstractSchemaRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/schema/AbstractSchemaRequest.java
index ed0f315b959..db6ef157c55 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/schema/AbstractSchemaRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/schema/AbstractSchemaRequest.java
@@ -16,11 +16,12 @@
  */
 package org.apache.solr.client.solrj.request.schema;
 
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.request.DataStoreSolrRequest;
 import org.apache.solr.common.params.SolrParams;
 
-public abstract class AbstractSchemaRequest<T extends SolrResponse> extends SolrRequest<T> {
+public abstract class AbstractSchemaRequest<T extends SolrResponse>
+    extends DataStoreSolrRequest<T> {
   private SolrParams params = null;
 
   public AbstractSchemaRequest(METHOD m, String path) {
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java b/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java
index 9d16525f496..fa83c1ddb71 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/util/ClientUtils.java
@@ -27,6 +27,8 @@ import java.util.Collection;
 import java.util.Date;
 import java.util.Map;
 import java.util.Map.Entry;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
 import org.apache.solr.common.cloud.Slice;
@@ -239,4 +241,16 @@ public class ClientUtils {
       target.put(key, slice);
     }
   }
+
+  /**
+   * Determines whether any SolrClient "default" collection should applied to the specified request
+   *
+   * @param providedCollection a collection/core explicitly provided to the SolrClient (typically
+   *     through {@link org.apache.solr.client.solrj.SolrClient#request(SolrRequest, String)}
+   * @param request the {@link SolrRequest} being executed
+   */
+  public static boolean shouldApplyDefaultDataStore(
+      String providedCollection, SolrRequest<? extends SolrResponse> request) {
+    return providedCollection == null && request.requiresDataStore();
+  }
 }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBuilderTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBuilderTest.java
index c6cd4779f96..c311b99a32b 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBuilderTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBuilderTest.java
@@ -77,7 +77,7 @@ public class ConcurrentUpdateSolrClientBuilderTest extends SolrTestCase {
   public void testDefaultCollectionPassedFromBuilderToClient() throws IOException {
     try (SolrClient createdClient =
         new ConcurrentUpdateSolrClient.Builder("someurl")
-            .withDefaultCollection("aCollection")
+            .withDefaultDataStore("aCollection")
             .build()) {
       assertEquals("aCollection", createdClient.getDefaultCollection());
     }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientBuilderTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientBuilderTest.java
index 6aba55d47bf..65e113a0a11 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientBuilderTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientBuilderTest.java
@@ -88,7 +88,7 @@ public class HttpSolrClientBuilderTest extends SolrTestCase {
   @Test
   public void testDefaultCollectionPassedFromBuilderToClient() throws IOException {
     try (final SolrClient createdClient =
-        new Builder(ANY_BASE_SOLR_URL).withDefaultCollection("aCollection").build()) {
+        new Builder(ANY_BASE_SOLR_URL).withDefaultDataStore("aCollection").build()) {
       assertEquals("aCollection", createdClient.getDefaultCollection());
     }
   }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/LBHttpSolrClientBuilderTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/LBHttpSolrClientBuilderTest.java
index 6db869cf96b..2aa7e9b756b 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/LBHttpSolrClientBuilderTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/LBHttpSolrClientBuilderTest.java
@@ -83,7 +83,7 @@ public class LBHttpSolrClientBuilderTest extends SolrTestCase {
     try (LBHttpSolrClient createdClient =
         new LBHttpSolrClient.Builder()
             .withBaseSolrUrl(ANY_BASE_SOLR_URL)
-            .withDefaultCollection("aCollection")
+            .withDefaultDataStore("aCollection")
             .build()) {
       assertEquals("aCollection", createdClient.getDefaultCollection());
     }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/util/ClientUtilsTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/util/ClientUtilsTest.java
index d5997f5d85e..d62c88e539d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/util/ClientUtilsTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/util/ClientUtilsTest.java
@@ -17,6 +17,9 @@
 package org.apache.solr.client.solrj.util;
 
 import org.apache.solr.SolrTestCase;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.junit.Test;
 
 /**
  * @since solr 1.3
@@ -30,4 +33,20 @@ public class ClientUtilsTest extends SolrTestCase {
     assertEquals("h\\:ello\\!", ClientUtils.escapeQueryChars("h:ello!"));
     assertEquals("h\\~\\!", ClientUtils.escapeQueryChars("h~!"));
   }
+
+  @Test
+  public void testDeterminesWhenToUseDefaultDataStore() {
+    final var noDefaultNeededRequest = new CollectionAdminRequest.List();
+    final var defaultNeededRequest = new UpdateRequest();
+
+    assertFalse(
+        "Expected default-coll to be skipped for collection-agnostic request",
+        ClientUtils.shouldApplyDefaultDataStore(null, noDefaultNeededRequest));
+    assertTrue(
+        "Expected default-coll to be used for collection-based request",
+        ClientUtils.shouldApplyDefaultDataStore(null, defaultNeededRequest));
+    assertFalse(
+        "Expected default-coll to be skipped when a collection is explicitly provided",
+        ClientUtils.shouldApplyDefaultDataStore("someCollection", defaultNeededRequest));
+  }
 }
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ConfigRequest.java b/solr/test-framework/src/java/org/apache/solr/cloud/ConfigRequest.java
index 92007134082..e4dbffe202e 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ConfigRequest.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ConfigRequest.java
@@ -19,6 +19,7 @@ package org.apache.solr.cloud;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.request.DataStoreSolrRequest;
 import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.response.SolrResponseBase;
 import org.apache.solr.common.params.CommonParams;
@@ -29,7 +30,7 @@ import org.apache.solr.common.params.SolrParams;
  * components, handlers, parsers, etc. to an otherwise generic configset.
  */
 @SuppressWarnings({"rawtypes"})
-public class ConfigRequest extends SolrRequest {
+public class ConfigRequest extends DataStoreSolrRequest {
 
   protected final String message;