You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by jd...@apache.org on 2024/03/12 17:08:03 UTC

(solr) branch main updated: SOLR-599 New Solr Client Using Java 11+ java.net.http.HttpClient (#2259)

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

jdyer 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 188ca8872cd SOLR-599 New Solr Client Using Java 11+ java.net.http.HttpClient (#2259)
188ca8872cd is described below

commit 188ca8872cdbe61ea229bc40fadee20e7ba50053
Author: James Dyer <jd...@apache.org>
AuthorDate: Tue Mar 12 12:07:57 2024 -0500

    SOLR-599 New Solr Client Using Java 11+ java.net.http.HttpClient (#2259)
    
    SOLR-599: new SolrJ Http Client using the JDK built-in java.net.http.HttpClient (#2259)
---
 .../randomization/policies/solr-tests.policy       |  46 ++
 solr/CHANGES.txt                                   |   2 +
 .../modules/deployment-guide/pages/solrj.adoc      |   1 +
 .../solr/client/solrj/impl/Http2SolrClient.java    | 720 ++++---------------
 .../solr/client/solrj/impl/HttpJdkSolrClient.java  | 515 ++++++++++++++
 .../solr/client/solrj/impl/HttpSolrClient.java     |   2 +-
 .../solr/client/solrj/impl/HttpSolrClientBase.java | 390 ++++++++++
 .../solrj/impl/HttpSolrClientBuilderBase.java      | 201 ++++++
 .../solr/client/solrj/impl/DebugServlet.java       | 161 +++++
 .../impl/Http2SolrClientCompatibilityTest.java     |   6 +-
 .../solrj/impl/Http2SolrClientProxyTest.java       |  73 +-
 .../client/solrj/impl/Http2SolrClientTest.java     | 785 ++++-----------------
 .../client/solrj/impl/HttpJdkSolrClientTest.java   | 599 ++++++++++++++++
 .../client/solrj/impl/HttpSolrClientTestBase.java  | 535 ++++++++++++++
 .../java/org/apache/solr/util/SSLTestConfig.java   |  15 +
 15 files changed, 2794 insertions(+), 1257 deletions(-)

diff --git a/gradle/testing/randomization/policies/solr-tests.policy b/gradle/testing/randomization/policies/solr-tests.policy
index c4b07f8ac1a..7a10a09a0cd 100644
--- a/gradle/testing/randomization/policies/solr-tests.policy
+++ b/gradle/testing/randomization/policies/solr-tests.policy
@@ -184,6 +184,52 @@ grant {
   // for Apache HttpClient useSystemProperties
   permission java.net.NetPermission "getProxySelector";
   permission java.net.NetPermission "requestPasswordAuthentication";
+
+  // for java.net.http.HttpClient.  See HttpJdkSolrClientTest
+  permission "java.net.URLPermission" "http://127.0.0.1:*/solr/debug/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://127.0.0.1:*/solr/debug/foo/admin/ping", "GET:*";
+  permission "java.net.URLPermission" "http://127.0.0.1:*/solr/redirect/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://127.0.0.1:*/solr/slow/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://127.0.0.1:*/solr/collection1/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://127.0.0.1:*/solr/collection1/admin/ping", "GET:*";
+  permission "java.net.URLPermission" "socket://127.0.0.1:*", "CONNECT:*";
+
+  permission "java.net.URLPermission" "https://127.0.0.1:*/solr/debug/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://127.0.0.1:*/solr/debug/foo/admin/ping", "GET:*";
+  permission "java.net.URLPermission" "https://127.0.0.1:*/solr/redirect/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://127.0.0.1:*/solr/slow/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://127.0.0.1:*/solr/collection1/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://127.0.0.1:*/solr/collection1/admin/ping", "GET:*";
+
+  permission "java.net.URLPermission" "http://localhost:*/solr/debug/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://localhost:*/solr/debug/foo/admin/ping", "GET:*";
+  permission "java.net.URLPermission" "http://localhost:*/solr/redirect/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://localhost:*/solr/slow/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://localhost:*/solr/collection1/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://localhost:*/solr/collection1/admin/ping", "GET:*";
+  permission "java.net.URLPermission" "socket://localhost:*", "CONNECT:*";
+
+  permission "java.net.URLPermission" "https://localhost:*/solr/debug/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://localhost:*/solr/debug/foo/admin/ping", "GET:*";
+  permission "java.net.URLPermission" "https://localhost:*/solr/redirect/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://localhost:*/solr/slow/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://localhost:*/solr/collection1/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://localhost:*/solr/collection1/admin/ping", "GET:*";
+
+  permission "java.net.URLPermission" "http://[::1]:*/solr/debug/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://[::1]:*/solr/debug/foo/admin/ping", "GET:*";
+  permission "java.net.URLPermission" "http://[::1]:*/solr/redirect/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://[::1]:*/solr/slow/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://[::1]:*/solr/collection1/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "http://[::1]:*/solr/collection1/admin/ping", "GET:*";
+  permission "java.net.URLPermission" "socket://[::1]:*", "CONNECT:*";
+
+  permission "java.net.URLPermission" "https://[::1]:*/solr/debug/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://[::1]:*/solr/debug/foo/admin/ping", "GET:*";
+  permission "java.net.URLPermission" "https://[::1]:*/solr/redirect/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://[::1]:*/solr/slow/foo/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://[::1]:*/solr/collection1/*", "HEAD,GET,PUT,POST:*";
+  permission "java.net.URLPermission" "https://[::1]:*/solr/collection1/admin/ping", "GET:*";
 };
 
 // additional permissions based on system properties set by /bin/solr
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 2c62ab74afe..84d0b79dbd7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -88,6 +88,8 @@ New Features
 ---------------------
 * SOLR-17141: Implement 'cpuAllowed' query parameter to limit the maximum CPU usage by a running query. (Andrzej Bialecki, Gus Heck, David Smiley)
 
+* SOLR-599: Add a new SolrJ client using the JDK’s built-in Http Client.  (James Dyer)
+
 Improvements
 ---------------------
 * SOLR-17119: When registering or updating a ConfigurablePlugin through the `/cluster/plugin` API,
diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/solrj.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/solrj.adoc
index bda99df4304..8b6997e7c64 100644
--- a/solr/solr-ref-guide/modules/deployment-guide/pages/solrj.adoc
+++ b/solr/solr-ref-guide/modules/deployment-guide/pages/solrj.adoc
@@ -99,6 +99,7 @@ Requests are sent in the form of {solr-javadocs}/solrj/org/apache/solr/client/so
 Communicates directly with a single Solr node.
 - {solr-javadocs}/solrj/org/apache/solr/client/solrj/impl/Http2SolrClient.html[`Http2SolrClient`] - async, non-blocking and general-purpose client that leverage HTTP/2.
 This class is experimental therefore its API's might change or be removed in minor versions of SolrJ.
+- {solr-javadocs}/solrj/org/apache/solr/client/solrj/impl/HttpJdkSolrClient.html[`HttpJdkSolrClient`] - General-purpose client using the JDK's built-in Http Client.  Supports both Http/2 and Http/1.1.  Targeted for those users wishing to minimize application dependencies.
 - {solr-javadocs}/solrj/org/apache/solr/client/solrj/impl/LBHttpSolrClient.html[`LBHttpSolrClient`] - balances request load across a list of Solr nodes.
 Adjusts the list of "in-service" nodes based on node health.
 - {solr-javadocs}/solrj/org/apache/solr/client/solrj/impl/LBHttp2SolrClient.html[`LBHttp2SolrClient`] - just like `LBHttpSolrClient` but using `Http2SolrClient` instead.
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 79de13f4a26..07264bd87a9 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
@@ -16,9 +16,6 @@
  */
 package org.apache.solr.client.solrj.impl;
 
-import static org.apache.solr.common.util.Utils.getObjectByPath;
-
-import java.io.ByteArrayOutputStream;
 import java.io.Closeable;
 import java.io.FilterInputStream;
 import java.io.IOException;
@@ -27,21 +24,13 @@ import java.lang.invoke.MethodHandles;
 import java.lang.reflect.InvocationTargetException;
 import java.net.ConnectException;
 import java.net.CookieStore;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Base64;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Phaser;
@@ -50,21 +39,17 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.stream.Collectors;
 import org.apache.solr.client.solrj.ResponseParser;
-import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.SSLConfig;
-import org.apache.solr.client.solrj.impl.BaseHttpSolrClient.RemoteExecutionException;
 import org.apache.solr.client.solrj.impl.BaseHttpSolrClient.RemoteSolrException;
 import org.apache.solr.client.solrj.impl.HttpListenerFactory.RequestResponseListener;
 import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.client.solrj.util.AsyncListener;
 import org.apache.solr.client.solrj.util.Cancellable;
 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;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.UpdateParams;
@@ -73,7 +58,6 @@ import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
-import org.apache.solr.common.util.Utils;
 import org.eclipse.jetty.client.HttpClient;
 import org.eclipse.jetty.client.HttpClientTransport;
 import org.eclipse.jetty.client.HttpProxy;
@@ -97,7 +81,6 @@ import org.eclipse.jetty.http.HttpField;
 import org.eclipse.jetty.http.HttpFields;
 import org.eclipse.jetty.http.HttpHeader;
 import org.eclipse.jetty.http.HttpMethod;
-import org.eclipse.jetty.http.HttpStatus;
 import org.eclipse.jetty.http.MimeTypes;
 import org.eclipse.jetty.http2.client.HTTP2Client;
 import org.eclipse.jetty.http2.client.http.HttpClientTransportOverHTTP2;
@@ -124,57 +107,30 @@ import org.slf4j.MDC;
  *
  * @lucene.experimental
  */
-public class Http2SolrClient extends SolrClient {
+public class Http2SolrClient extends HttpSolrClientBase {
   public static final String REQ_PRINCIPAL_KEY = "solr-req-principal";
 
   private static volatile SSLConfig defaultSSLConfig;
-
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final String AGENT = "Solr[" + Http2SolrClient.class.getName() + "] 2.0";
-  private static final Charset FALLBACK_CHARSET = StandardCharsets.UTF_8;
-  private static final String DEFAULT_PATH = "/select";
-  private static final List<String> errPath = Arrays.asList("metadata", "error-class");
 
   private final HttpClient httpClient;
-  private final Set<String> urlParamNames;
-  private final long idleTimeoutMillis;
-  private final long requestTimeoutMillis;
 
-  // updating parser instance needs to go via the setter to ensure update of defaultParserMimeTypes
-  private ResponseParser parser = new BinaryResponseParser();
-  private Set<String> defaultParserMimeTypes;
+  private SSLConfig sslConfig;
 
-  protected RequestWriter requestWriter = new BinaryRequestWriter();
   private List<HttpListenerFactory> listenerFactory = new ArrayList<>();
   private final AsyncTracker asyncTracker = new AsyncTracker();
 
-  /** The URL of the Solr server. */
-  private final String serverBaseUrl;
-
   private final boolean closeClient;
   private ExecutorService executor;
   private boolean shutdownExecutor;
 
-  final String basicAuthAuthorizationStr;
   private AuthenticationStoreHolder authenticationStore;
 
   private KeyStoreScanner scanner;
 
   protected Http2SolrClient(String serverBaseUrl, Builder builder) {
-    if (serverBaseUrl != null) {
-      if (!serverBaseUrl.equals("/") && serverBaseUrl.endsWith("/")) {
-        serverBaseUrl = serverBaseUrl.substring(0, serverBaseUrl.length() - 1);
-      }
-
-      if (serverBaseUrl.startsWith("//")) {
-        serverBaseUrl = serverBaseUrl.substring(1, serverBaseUrl.length());
-      }
-      this.serverBaseUrl = serverBaseUrl;
-    } else {
-      this.serverBaseUrl = null;
-    }
-
-    this.idleTimeoutMillis = builder.idleTimeoutMillis;
+    super(serverBaseUrl, builder);
 
     if (builder.httpClient != null) {
       this.httpClient = builder.httpClient;
@@ -183,26 +139,11 @@ public class Http2SolrClient extends SolrClient {
       this.httpClient = createHttpClient(builder);
       this.closeClient = true;
     }
-    this.basicAuthAuthorizationStr = builder.basicAuthAuthorizationStr;
-    if (builder.requestWriter != null) {
-      this.requestWriter = builder.requestWriter;
-    }
-    if (builder.responseParser != null) {
-      this.parser = builder.responseParser;
-    }
+
     updateDefaultMimeTypeForParser();
-    this.defaultCollection = builder.defaultCollection;
-    if (builder.requestTimeoutMillis != null) {
-      this.requestTimeoutMillis = builder.requestTimeoutMillis;
-    } else {
-      this.requestTimeoutMillis = -1;
-    }
+
     this.httpClient.setFollowRedirects(Boolean.TRUE.equals(builder.followRedirects));
-    if (builder.urlParamNames != null) {
-      this.urlParamNames = builder.urlParamNames;
-    } else {
-      this.urlParamNames = Set.of();
-    }
+
     assert ObjectReleaseTracker.track(this);
   }
 
@@ -376,14 +317,6 @@ public class Http2SolrClient extends SolrClient {
     this.authenticationStore.updateAuthenticationStore(authenticationStore);
   }
 
-  public boolean isV2ApiRequest(final SolrRequest<?> request) {
-    return request instanceof V2Request || request.getPath().contains("/____v2");
-  }
-
-  public long getIdleTimeout() {
-    return idleTimeoutMillis;
-  }
-
   public static class OutStream implements Closeable {
     private final String origCollection;
     private final ModifiableSolrParams origParams;
@@ -436,12 +369,8 @@ public class Http2SolrClient extends SolrClient {
       throws IOException {
     String contentType = requestWriter.getUpdateContentType();
     final ModifiableSolrParams origParams = new ModifiableSolrParams(updateRequest.getParams());
-
-    // The parser 'wt=' and 'version=' params are used instead of the
-    // original params
-    ModifiableSolrParams requestParams = new ModifiableSolrParams(origParams);
-    requestParams.set(CommonParams.WT, parser.getWriterType());
-    requestParams.set(CommonParams.VERSION, parser.getVersion());
+    ModifiableSolrParams requestParams =
+        initalizeSolrParams(updateRequest, responseParser(updateRequest));
 
     String basePath = baseUrl;
     if (collection != null) basePath += "/" + collection;
@@ -555,8 +484,9 @@ public class Http2SolrClient extends SolrClient {
   @Override
   public NamedList<Object> request(SolrRequest<?> solrRequest, String collection)
       throws SolrServerException, IOException {
-    if (ClientUtils.shouldApplyDefaultCollection(collection, solrRequest))
+    if (ClientUtils.shouldApplyDefaultCollection(collection, solrRequest)) {
       collection = defaultCollection;
+    }
     String url = getRequestPath(solrRequest, collection);
     Throwable abortCause = null;
     Request req = null;
@@ -609,8 +539,17 @@ public class Http2SolrClient extends SolrClient {
       mimeType = MimeTypes.getContentTypeWithoutCharset(contentType);
       encoding = MimeTypes.getCharsetFromContentType(contentType);
     }
+    String responseMethod = response.getRequest() == null ? "" : response.getRequest().getMethod();
     return processErrorsAndResponse(
-        response, parser, is, mimeType, encoding, isV2ApiRequest(solrRequest), urlExceptionMessage);
+        response.getStatus(),
+        response.getReason(),
+        responseMethod,
+        parser,
+        is,
+        mimeType,
+        encoding,
+        isV2ApiRequest(solrRequest),
+        urlExceptionMessage);
   }
 
   private void setBasicAuthHeader(SolrRequest<?> solrRequest, Request req) {
@@ -624,11 +563,6 @@ public class Http2SolrClient extends SolrClient {
     }
   }
 
-  static String basicAuthCredentialsToAuthorizationString(String user, String pass) {
-    String userPass = user + ":" + pass;
-    return "Basic " + Base64.getEncoder().encodeToString(userPass.getBytes(FALLBACK_CHARSET));
-  }
-
   private void decorateRequest(Request req, SolrRequest<?> solrRequest, boolean isAsync) {
     req.headers(headers -> headers.remove(HttpHeader.ACCEPT_ENCODING));
 
@@ -663,53 +597,14 @@ public class Http2SolrClient extends SolrClient {
     }
   }
 
-  private String changeV2RequestEndpoint(String basePath) throws MalformedURLException {
-    URL oldURL = new URL(basePath);
-    String newPath = oldURL.getPath().replaceFirst("/solr", "/api");
-    return new URL(oldURL.getProtocol(), oldURL.getHost(), oldURL.getPort(), newPath).toString();
-  }
-
-  private String getRequestPath(SolrRequest<?> solrRequest, String collection)
-      throws MalformedURLException {
-    String basePath = solrRequest.getBasePath() == null ? serverBaseUrl : solrRequest.getBasePath();
-    if (collection != null) basePath += "/" + collection;
-
-    if (solrRequest instanceof V2Request) {
-      if (System.getProperty("solr.v2RealPath") == null) {
-        basePath = changeV2RequestEndpoint(basePath);
-      } else {
-        basePath = serverBaseUrl + "/____v2";
-      }
-    }
-    String path = requestWriter.getPath(solrRequest);
-    if (path == null || !path.startsWith("/")) {
-      path = DEFAULT_PATH;
-    }
-
-    return basePath + path;
-  }
-
   private Request makeRequestAndSend(
       SolrRequest<?> solrRequest, String url, InputStreamResponseListener listener, boolean isAsync)
       throws IOException, SolrServerException {
 
-    // TODO add invariantParams support
-    ResponseParser parser =
-        solrRequest.getResponseParser() == null ? this.parser : solrRequest.getResponseParser();
-
-    // The parser 'wt=' and 'version=' params are used instead of the original
-    // params
-    ModifiableSolrParams wparams = new ModifiableSolrParams(solrRequest.getParams());
-    wparams.set(CommonParams.WT, parser.getWriterType());
-    wparams.set(CommonParams.VERSION, parser.getVersion());
+    ModifiableSolrParams wparams = initalizeSolrParams(solrRequest, responseParser(solrRequest));
 
     if (SolrRequest.METHOD.GET == solrRequest.getMethod()) {
-      RequestWriter.ContentWriter contentWriter = requestWriter.getContentWriter(solrRequest);
-      Collection<ContentStream> streams =
-          contentWriter == null ? requestWriter.getContentStreams(solrRequest) : null;
-      if (contentWriter != null || streams != null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "GET can't send streams!");
-      }
+      validateGetRequest(solrRequest);
       var r = httpClient.newRequest(url + wparams.toQueryString()).method(HttpMethod.GET);
       decorateRequest(r, solrRequest, isAsync);
       r.send(listener);
@@ -729,12 +624,7 @@ public class Http2SolrClient extends SolrClient {
       Collection<ContentStream> streams =
           contentWriter == null ? requestWriter.getContentStreams(solrRequest) : null;
 
-      boolean isMultipart = false;
-      if (streams != null) {
-        boolean hasNullStreamName = false;
-        hasNullStreamName = streams.stream().anyMatch(cs -> cs.getName() == null);
-        isMultipart = !hasNullStreamName && streams.size() > 1;
-      }
+      boolean isMultipart = isMultipart(streams);
 
       HttpMethod method =
           SolrRequest.METHOD.POST == solrRequest.getMethod() ? HttpMethod.POST : HttpMethod.PUT;
@@ -760,7 +650,7 @@ public class Http2SolrClient extends SolrClient {
         return r;
 
       } else {
-        // It is has one stream, it is the post body, put the params in the URL
+        // If is has one stream, it is the post body, put the params in the URL
         ContentStream contentStream = streams.iterator().next();
         var content =
             new InputStreamRequestContent(
@@ -834,188 +724,40 @@ public class Http2SolrClient extends SolrClient {
     return req;
   }
 
-  private boolean wantStream(final ResponseParser processor) {
-    return processor == null || processor instanceof InputStreamResponseParser;
+  @Override
+  protected boolean isFollowRedirects() {
+    return httpClient.isFollowRedirects();
   }
 
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  private NamedList<Object> processErrorsAndResponse(
-      Response response,
-      final ResponseParser processor,
-      InputStream is,
-      String mimeType,
-      String encoding,
-      final boolean isV2Api,
-      String urlExceptionMessage)
-      throws SolrServerException {
-    boolean shouldClose = true;
-    try {
-      // handle some http level checks before trying to parse the response
-      int httpStatus = response.getStatus();
-
-      switch (httpStatus) {
-        case HttpStatus.OK_200:
-        case HttpStatus.BAD_REQUEST_400:
-        case HttpStatus.CONFLICT_409:
-          break;
-        case HttpStatus.MOVED_PERMANENTLY_301:
-        case HttpStatus.MOVED_TEMPORARILY_302:
-          if (!httpClient.isFollowRedirects()) {
-            throw new SolrServerException(
-                "Server at " + urlExceptionMessage + " sent back a redirect (" + httpStatus + ").");
-          }
-          break;
-        default:
-          if (processor == null || mimeType == null) {
-            throw new RemoteSolrException(
-                urlExceptionMessage,
-                httpStatus,
-                "non ok status: " + httpStatus + ", message:" + response.getReason(),
-                null);
-          }
-      }
-
-      if (wantStream(processor)) {
-        // no processor specified, return raw stream
-        NamedList<Object> rsp = new NamedList<>();
-        rsp.add("stream", is);
-        rsp.add("responseStatus", httpStatus);
-        // Only case where stream should not be closed
-        shouldClose = false;
-        return rsp;
-      }
-
-      checkContentType(processor, is, mimeType, encoding, httpStatus, urlExceptionMessage);
+  @Override
+  protected boolean processorAcceptsMimeType(
+      Collection<String> processorSupportedContentTypes, String mimeType) {
 
-      NamedList<Object> rsp;
-      try {
-        rsp = processor.processResponse(is, encoding);
-      } catch (Exception e) {
-        throw new RemoteSolrException(urlExceptionMessage, httpStatus, e.getMessage(), e);
-      }
+    return processorSupportedContentTypes.stream()
+        .map(ct -> MimeTypes.getContentTypeWithoutCharset(ct).trim())
+        .anyMatch(mimeType::equalsIgnoreCase);
+  }
 
-      Object error = rsp == null ? null : rsp.get("error");
-      if (rsp != null && error == null && processor instanceof NoOpResponseParser) {
-        error = rsp.get("response");
-      }
-      if (error != null
-          && (String.valueOf(getObjectByPath(error, true, errPath))
-              .endsWith("ExceptionWithErrObject"))) {
-        throw RemoteExecutionException.create(urlExceptionMessage, rsp);
-      }
-      if (httpStatus != HttpStatus.OK_200 && !isV2Api) {
-        NamedList<String> metadata = null;
-        String reason = null;
-        try {
-          if (error != null) {
-            reason = (String) Utils.getObjectByPath(error, false, Collections.singletonList("msg"));
-            if (reason == null) {
-              reason =
-                  (String) Utils.getObjectByPath(error, false, Collections.singletonList("trace"));
-            }
-            Object metadataObj =
-                Utils.getObjectByPath(error, false, Collections.singletonList("metadata"));
-            if (metadataObj instanceof NamedList) {
-              metadata = (NamedList<String>) metadataObj;
-            } else if (metadataObj instanceof List) {
-              // NamedList parsed as List convert to NamedList again
-              List<Object> list = (List<Object>) metadataObj;
-              metadata = new NamedList<>(list.size() / 2);
-              for (int i = 0; i < list.size(); i += 2) {
-                metadata.add((String) list.get(i), (String) list.get(i + 1));
-              }
-            } else if (metadataObj instanceof Map) {
-              metadata = new NamedList((Map) metadataObj);
-            }
-          }
-        } catch (Exception ex) {
-          /* Ignored */
-        }
-        if (reason == null) {
-          StringBuilder msg = new StringBuilder();
-          msg.append(response.getReason())
-              .append("\n")
-              .append("request: ")
-              .append(response.getRequest().getMethod());
-          if (error != null) {
-            msg.append("\n\nError returned:\n").append(error);
-          }
-          reason = java.net.URLDecoder.decode(msg.toString(), FALLBACK_CHARSET);
-        }
-        RemoteSolrException rss =
-            new RemoteSolrException(urlExceptionMessage, httpStatus, reason, null);
-        if (metadata != null) rss.setMetadata(metadata);
-        throw rss;
-      }
-      return rsp;
-    } finally {
-      if (shouldClose) {
-        try {
-          is.close();
-        } catch (IOException e) {
-          // quitely
-        }
-      }
-    }
+  @Override
+  protected void updateDefaultMimeTypeForParser() {
+    defaultParserMimeTypes =
+        parser.getContentTypes().stream()
+            .map(ct -> MimeTypes.getContentTypeWithoutCharset(ct).trim().toLowerCase(Locale.ROOT))
+            .collect(Collectors.toSet());
   }
 
-  /**
-   * Validates that the content type in the response can be processed by the Response Parser. Throws
-   * a {@code RemoteSolrException} if not.
-   */
-  private void checkContentType(
-      ResponseParser processor,
-      InputStream is,
-      String mimeType,
-      String encoding,
-      int httpStatus,
-      String urlExceptionMessage) {
-    if (mimeType == null
-        || (processor == this.parser && defaultParserMimeTypes.contains(mimeType))) {
-      // Shortcut the default scenario
-      return;
-    }
-    final Collection<String> processorSupportedContentTypes = processor.getContentTypes();
-    if (processorSupportedContentTypes != null && !processorSupportedContentTypes.isEmpty()) {
-      boolean processorAcceptsMimeType =
-          processorSupportedContentTypes.stream()
-              .map(ct -> MimeTypes.getContentTypeWithoutCharset(ct).trim())
-              .anyMatch(mimeType::equalsIgnoreCase);
-      if (!processorAcceptsMimeType) {
-        // unexpected mime type
-        final String allSupportedTypes =
-            processorSupportedContentTypes.stream()
-                .map(
-                    ct ->
-                        MimeTypes.getContentTypeWithoutCharset(ct).trim().toLowerCase(Locale.ROOT))
-                .collect(Collectors.joining(", "));
-        String prefix =
-            "Expected mime type in [" + allSupportedTypes + "] but got " + mimeType + ". ";
-        String exceptionEncoding = encoding != null ? encoding : FALLBACK_CHARSET.name();
-        try {
-          ByteArrayOutputStream body = new ByteArrayOutputStream();
-          is.transferTo(body);
-          throw new RemoteSolrException(
-              urlExceptionMessage, httpStatus, prefix + body.toString(exceptionEncoding), null);
-        } catch (IOException e) {
-          throw new RemoteSolrException(
-              urlExceptionMessage,
-              httpStatus,
-              "Could not parse response with encoding " + exceptionEncoding,
-              e);
-        }
-      }
-    }
+  @Override
+  protected String allProcessorSupportedContentTypesCommaDelimited(
+      Collection<String> processorSupportedContentTypes) {
+    return processorSupportedContentTypes.stream()
+        .map(ct -> MimeTypes.getContentTypeWithoutCharset(ct).trim().toLowerCase(Locale.ROOT))
+        .collect(Collectors.joining(", "));
   }
 
   protected RequestWriter getRequestWriter() {
     return requestWriter;
   }
 
-  public String getBaseURL() {
-    return serverBaseUrl;
-  }
-
   private static class AsyncTracker {
     private static final int MAX_OUTSTANDING_REQUESTS = 1000;
 
@@ -1057,31 +799,20 @@ public class Http2SolrClient extends SolrClient {
     }
   }
 
-  public static class Builder {
+  public static class Builder
+      extends HttpSolrClientBuilderBase<Http2SolrClient.Builder, Http2SolrClient> {
 
     private HttpClient httpClient;
-    private SSLConfig sslConfig = defaultSSLConfig;
-    private Long idleTimeoutMillis;
-    private Long connectionTimeoutMillis;
-    private Long requestTimeoutMillis;
-    private Integer maxConnectionsPerHost;
-    private String basicAuthAuthorizationStr;
-    private boolean useHttp1_1 = Boolean.getBoolean("solr.http1");
-    private Boolean followRedirects;
-    protected String baseSolrUrl;
-    private ExecutorService executor;
-    protected RequestWriter requestWriter;
-    protected ResponseParser responseParser;
-    protected String defaultCollection;
-    private Set<String> urlParamNames;
-    private CookieStore cookieStore = getDefaultCookieStore();
-    private String proxyHost;
-    private int proxyPort;
-    private boolean proxyIsSocks4;
-    private boolean proxyIsSecure;
-    private Long keyStoreReloadIntervalSecs;
-
-    public Builder() {}
+
+    protected CookieStore cookieStore;
+
+    private SSLConfig sslConfig;
+
+    protected Long keyStoreReloadIntervalSecs;
+
+    public Builder() {
+      super();
+    }
 
     /**
      * Initialize a Builder object, based on the provided Solr URL.
@@ -1099,10 +830,99 @@ public class Http2SolrClient extends SolrClient {
      *     created clients.
      */
     public Builder(String baseSolrUrl) {
+      super();
       this.baseSolrUrl = baseSolrUrl;
     }
 
+    public HttpSolrClientBuilderBase<Http2SolrClient.Builder, Http2SolrClient> withSSLConfig(
+        SSLConfig sslConfig) {
+      this.sslConfig = sslConfig;
+      return this;
+    }
+
+    /**
+     * Set maxConnectionsPerHost for http1 connections, maximum number http2 connections is limited
+     * to 4
+     *
+     * @deprecated Please use {@link #withMaxConnectionsPerHost(int)}
+     */
+    @Deprecated(since = "9.2")
+    public Http2SolrClient.Builder maxConnectionsPerHost(int max) {
+      withMaxConnectionsPerHost(max);
+      return this;
+    }
+
+    /**
+     * Set the scanning interval to check for updates in the Key Store used by this client. If the
+     * interval is unset, 0 or less, then the Key Store Scanner is not created, and the client will
+     * not attempt to update key stores. The minimum value between checks is 1 second.
+     *
+     * @param interval Interval between checks
+     * @param unit The unit for the interval
+     * @return This builder
+     */
+    public Http2SolrClient.Builder withKeyStoreReloadInterval(long interval, TimeUnit unit) {
+      this.keyStoreReloadIntervalSecs = unit.toSeconds(interval);
+      if (this.keyStoreReloadIntervalSecs == 0 && interval > 0) {
+        this.keyStoreReloadIntervalSecs = 1L;
+      }
+      return this;
+    }
+
+    /**
+     * @deprecated Please use {@link #withIdleTimeout(long, TimeUnit)}
+     */
+    @Deprecated(since = "9.2")
+    public Http2SolrClient.Builder idleTimeout(int idleConnectionTimeout) {
+      withIdleTimeout(idleConnectionTimeout, TimeUnit.MILLISECONDS);
+      return this;
+    }
+
+    /**
+     * @deprecated Please use {@link #withConnectionTimeout(long, TimeUnit)}
+     */
+    @Deprecated(since = "9.2")
+    public Http2SolrClient.Builder connectionTimeout(int connectionTimeout) {
+      withConnectionTimeout(connectionTimeout, TimeUnit.MILLISECONDS);
+      return this;
+    }
+
+    /**
+     * Set a timeout in milliseconds for requests issued by this client.
+     *
+     * @param requestTimeout The timeout in milliseconds
+     * @return this Builder.
+     * @deprecated Please use {@link #withRequestTimeout(long, TimeUnit)}
+     */
+    @Deprecated(since = "9.2")
+    public Http2SolrClient.Builder requestTimeout(int requestTimeout) {
+      withRequestTimeout(requestTimeout, TimeUnit.MILLISECONDS);
+      return this;
+    }
+
+    private CookieStore getDefaultCookieStore() {
+      if (Boolean.getBoolean("solr.http.disableCookies")) {
+        return new HttpCookieStore.Empty();
+      }
+      /*
+       * We could potentially have a Supplier<CookieStore> if we ever need further customization support,
+       * but for now it's only either Empty or default (in-memory).
+       */
+      return null;
+    }
+
+    protected <B extends HttpSolrClientBase> B build(Class<B> type) {
+      return type.cast(build());
+    }
+
+    @Override
     public Http2SolrClient build() {
+      if (sslConfig == null) {
+        sslConfig = Http2SolrClient.defaultSSLConfig;
+      }
+      if (cookieStore == null) {
+        cookieStore = getDefaultCookieStore();
+      }
       if (idleTimeoutMillis == null || idleTimeoutMillis <= 0) {
         idleTimeoutMillis = (long) HttpClientUtil.DEFAULT_SO_TIMEOUT;
       }
@@ -1158,17 +978,6 @@ public class Http2SolrClient extends SolrClient {
       }
     }
 
-    private static CookieStore getDefaultCookieStore() {
-      if (Boolean.getBoolean("solr.http.disableCookies")) {
-        return new HttpCookieStore.Empty();
-      }
-      /*
-       * We could potentially have a Supplier<CookieStore> if we ever need further customization support,
-       * but for now it's only either Empty or default (in-memory).
-       */
-      return null;
-    }
-
     /**
      * Provide a seed Http2SolrClient for the builder values, values can still be overridden by
      * using builder methods
@@ -1200,166 +1009,6 @@ public class Http2SolrClient extends SolrClient {
       return this;
     }
 
-    /** Provides a {@link RequestWriter} for created clients to use when handing requests. */
-    public Builder withRequestWriter(RequestWriter requestWriter) {
-      this.requestWriter = requestWriter;
-      return this;
-    }
-
-    /** Provides a {@link ResponseParser} for created clients to use when handling requests. */
-    public Builder withResponseParser(ResponseParser responseParser) {
-      this.responseParser = responseParser;
-      return this;
-    }
-
-    /** Sets a default for core or collection based requests. */
-    public Builder withDefaultCollection(String defaultCoreOrCollection) {
-      this.defaultCollection = defaultCoreOrCollection;
-      return this;
-    }
-
-    public Builder withFollowRedirects(boolean followRedirects) {
-      this.followRedirects = followRedirects;
-      return this;
-    }
-
-    public Builder withExecutor(ExecutorService executor) {
-      this.executor = executor;
-      return this;
-    }
-
-    public Builder withSSLConfig(SSLConfig sslConfig) {
-      this.sslConfig = sslConfig;
-      return this;
-    }
-
-    public Builder withBasicAuthCredentials(String user, String pass) {
-      if (user != null || pass != null) {
-        if (user == null || pass == null) {
-          throw new IllegalStateException(
-              "Invalid Authentication credentials. Either both username and password or none must be provided");
-        }
-      }
-      this.basicAuthAuthorizationStr = basicAuthCredentialsToAuthorizationString(user, pass);
-      return this;
-    }
-
-    /**
-     * Expert Method
-     *
-     * @param urlParamNames set of param keys that are only sent via the query string. Note that the
-     *     param will be sent as a query string if the key is part of this Set or the SolrRequest's
-     *     query params.
-     * @see org.apache.solr.client.solrj.SolrRequest#getQueryParams
-     */
-    public Builder withTheseParamNamesInTheUrl(Set<String> urlParamNames) {
-      this.urlParamNames = urlParamNames;
-      return this;
-    }
-
-    /**
-     * Set maxConnectionsPerHost for http1 connections, maximum number http2 connections is limited
-     * to 4
-     *
-     * @deprecated Please use {@link #withMaxConnectionsPerHost(int)}
-     */
-    @Deprecated(since = "9.2")
-    public Builder maxConnectionsPerHost(int max) {
-      withMaxConnectionsPerHost(max);
-      return this;
-    }
-
-    /**
-     * Set maxConnectionsPerHost for http1 connections, maximum number http2 connections is limited
-     * to 4
-     */
-    public Builder withMaxConnectionsPerHost(int max) {
-      this.maxConnectionsPerHost = max;
-      return this;
-    }
-
-    /**
-     * Set the scanning interval to check for updates in the Key Store used by this client. If the
-     * interval is unset, 0 or less, then the Key Store Scanner is not created, and the client will
-     * not attempt to update key stores. The minimum value between checks is 1 second.
-     *
-     * @param interval Interval between checks
-     * @param unit The unit for the interval
-     * @return This builder
-     */
-    public Builder withKeyStoreReloadInterval(long interval, TimeUnit unit) {
-      this.keyStoreReloadIntervalSecs = unit.toSeconds(interval);
-      if (this.keyStoreReloadIntervalSecs == 0 && interval > 0) {
-        this.keyStoreReloadIntervalSecs = 1L;
-      }
-      return this;
-    }
-
-    /**
-     * @deprecated Please use {@link #withIdleTimeout(long, TimeUnit)}
-     */
-    @Deprecated(since = "9.2")
-    public Builder idleTimeout(int idleConnectionTimeout) {
-      withIdleTimeout(idleConnectionTimeout, TimeUnit.MILLISECONDS);
-      return this;
-    }
-
-    public Builder withIdleTimeout(long idleConnectionTimeout, TimeUnit unit) {
-      this.idleTimeoutMillis = TimeUnit.MILLISECONDS.convert(idleConnectionTimeout, unit);
-      return this;
-    }
-
-    public Long getIdleTimeoutMillis() {
-      return idleTimeoutMillis;
-    }
-
-    public Builder useHttp1_1(boolean useHttp1_1) {
-      this.useHttp1_1 = useHttp1_1;
-      return this;
-    }
-
-    /**
-     * @deprecated Please use {@link #withConnectionTimeout(long, TimeUnit)}
-     */
-    @Deprecated(since = "9.2")
-    public Builder connectionTimeout(int connectionTimeout) {
-      withConnectionTimeout(connectionTimeout, TimeUnit.MILLISECONDS);
-      return this;
-    }
-
-    public Builder withConnectionTimeout(long connectionTimeout, TimeUnit unit) {
-      this.connectionTimeoutMillis = TimeUnit.MILLISECONDS.convert(connectionTimeout, unit);
-      return this;
-    }
-
-    public Long getConnectionTimeout() {
-      return connectionTimeoutMillis;
-    }
-
-    /**
-     * Set a timeout in milliseconds for requests issued by this client.
-     *
-     * @deprecated Please use {@link #withRequestTimeout(long, TimeUnit)}
-     * @param requestTimeout The timeout in milliseconds
-     * @return this Builder.
-     */
-    @Deprecated(since = "9.2")
-    public Builder requestTimeout(int requestTimeout) {
-      withRequestTimeout(requestTimeout, TimeUnit.MILLISECONDS);
-      return this;
-    }
-
-    /**
-     * Set a timeout in milliseconds for requests issued by this client.
-     *
-     * @param requestTimeout The timeout in milliseconds
-     * @return this Builder.
-     */
-    public Builder withRequestTimeout(long requestTimeout, TimeUnit unit) {
-      this.requestTimeoutMillis = TimeUnit.MILLISECONDS.convert(requestTimeout, unit);
-      return this;
-    }
-
     /**
      * Set a cookieStore other than the default ({@code java.net.InMemoryCookieStore})
      *
@@ -1370,81 +1019,6 @@ public class Http2SolrClient extends SolrClient {
       this.cookieStore = cookieStore;
       return this;
     }
-
-    /**
-     * Setup a proxy
-     *
-     * @param host The proxy host
-     * @param port The proxy port
-     * @param isSocks4 If true creates an SOCKS 4 proxy, otherwise creates an HTTP proxy
-     * @param isSecure If true enables the secure flag on the proxy
-     * @return this Builder
-     */
-    public Builder withProxyConfiguration(
-        String host, int port, boolean isSocks4, boolean isSecure) {
-      this.proxyHost = host;
-      this.proxyPort = port;
-      this.proxyIsSocks4 = isSocks4;
-      this.proxyIsSecure = isSecure;
-      return this;
-    }
-
-    /**
-     * Setup basic authentication from a string formatted as username:password. If the string is
-     * Null then it doesn't do anything.
-     *
-     * @param credentials The username and password formatted as username:password
-     * @return this Builder
-     */
-    public Builder withOptionalBasicAuthCredentials(String credentials) {
-      if (credentials != null) {
-        if (credentials.indexOf(':') == -1) {
-          throw new IllegalStateException(
-              "Invalid Authentication credential formatting. Provide username and password in the 'username:password' format.");
-        }
-        String username = credentials.substring(0, credentials.indexOf(':'));
-        String password = credentials.substring(credentials.indexOf(':') + 1, credentials.length());
-        withBasicAuthCredentials(username, password);
-      }
-      return this;
-    }
-  }
-
-  public Set<String> getUrlParamNames() {
-    return urlParamNames;
-  }
-
-  private ModifiableSolrParams calculateQueryParams(
-      Set<String> queryParamNames, ModifiableSolrParams wparams) {
-    ModifiableSolrParams queryModParams = new ModifiableSolrParams();
-    if (queryParamNames != null) {
-      for (String param : queryParamNames) {
-        String[] value = wparams.getParams(param);
-        if (value != null) {
-          for (String v : value) {
-            queryModParams.add(param, v);
-          }
-          wparams.remove(param);
-        }
-      }
-    }
-    return queryModParams;
-  }
-
-  public ResponseParser getParser() {
-    return parser;
-  }
-
-  protected void setParser(ResponseParser parser) {
-    this.parser = parser;
-    updateDefaultMimeTypeForParser();
-  }
-
-  protected void updateDefaultMimeTypeForParser() {
-    defaultParserMimeTypes =
-        parser.getContentTypes().stream()
-            .map(ct -> MimeTypes.getContentTypeWithoutCharset(ct).trim().toLowerCase(Locale.ROOT))
-            .collect(Collectors.toSet());
   }
 
   public static void setDefaultSSLConfig(SSLConfig sslConfig) {
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpJdkSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpJdkSolrClient.java
new file mode 100644
index 00000000000..63d1d3ca207
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpJdkSolrClient.java
@@ -0,0 +1,515 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.impl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
+import java.lang.invoke.MethodHandles;
+import java.net.CookieHandler;
+import java.net.InetSocketAddress;
+import java.net.ProxySelector;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.http.HttpClient;
+import java.net.http.HttpRequest;
+import java.net.http.HttpResponse;
+import java.net.http.HttpTimeoutException;
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import javax.net.ssl.SSLContext;
+import org.apache.solr.client.solrj.ResponseParser;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.util.ClientUtils;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.ObjectReleaseTracker;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A SolrClient implementation that communicates to a Solr server using the built-in Java 11+ Http
+ * Client. This client is targeted for those users who wish to minimize application dependencies.
+ * This client will connect to solr using Http/2 but can seamlessly downgrade to Http/1.1 when
+ * connecting to Solr hosts running on older versions.
+ */
+public class HttpJdkSolrClient extends HttpSolrClientBase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String USER_AGENT =
+      "Solr[" + MethodHandles.lookup().lookupClass().getName() + "] 1.0";
+
+  protected HttpClient httpClient;
+
+  protected ExecutorService executor;
+
+  private boolean forceHttp11;
+
+  private boolean shutdownExecutor;
+
+  protected HttpJdkSolrClient(String serverBaseUrl, HttpJdkSolrClient.Builder builder) {
+    super(serverBaseUrl, builder);
+
+    HttpClient.Redirect followRedirects =
+        Boolean.TRUE.equals(builder.followRedirects)
+            ? HttpClient.Redirect.NORMAL
+            : HttpClient.Redirect.NEVER;
+    HttpClient.Builder b = HttpClient.newBuilder().followRedirects(followRedirects);
+    if (builder.sslContext != null) {
+      b.sslContext(builder.sslContext);
+    }
+
+    if (builder.executor != null) {
+      this.executor = builder.executor;
+      this.shutdownExecutor = false;
+    } else {
+      BlockingQueue<Runnable> queue = new LinkedBlockingQueue<>(1024);
+      this.executor =
+          new ExecutorUtil.MDCAwareThreadPoolExecutor(
+              4,
+              256,
+              60,
+              TimeUnit.SECONDS,
+              queue,
+              new SolrNamedThreadFactory(this.getClass().getSimpleName()));
+      this.shutdownExecutor = true;
+    }
+    b.executor(this.executor);
+
+    if (builder.useHttp1_1) {
+      this.forceHttp11 = true;
+      b.version(HttpClient.Version.HTTP_1_1);
+    }
+
+    if (builder.cookieHandler != null) {
+      b.cookieHandler(builder.cookieHandler);
+    }
+
+    if (builder.proxyHost != null) {
+      if (builder.proxyIsSocks4) {
+        log.warn(
+            "Socks4 is likely not supported by this client.  See https://bugs.openjdk.org/browse/JDK-8214516");
+      }
+      b.proxy(ProxySelector.of(new InetSocketAddress(builder.proxyHost, builder.proxyPort)));
+    }
+    this.httpClient = b.build();
+    updateDefaultMimeTypeForParser();
+
+    assert ObjectReleaseTracker.track(this);
+  }
+
+  @Override
+  public NamedList<Object> request(SolrRequest<?> solrRequest, String collection)
+      throws SolrServerException, IOException {
+    checkClosed();
+    if (ClientUtils.shouldApplyDefaultCollection(collection, solrRequest)) {
+      collection = defaultCollection;
+    }
+    String url = getRequestPath(solrRequest, collection);
+    ResponseParser parserToUse = responseParser(solrRequest);
+    ModifiableSolrParams queryParams = initalizeSolrParams(solrRequest, parserToUse);
+    HttpResponse<InputStream> resp = null;
+    try {
+      var reqb = HttpRequest.newBuilder();
+      switch (solrRequest.getMethod()) {
+        case GET:
+          {
+            resp = doGet(url, reqb, solrRequest, queryParams);
+            break;
+          }
+        case POST:
+        case PUT:
+          {
+            resp = doPutOrPost(url, solrRequest.getMethod(), reqb, solrRequest, queryParams);
+            break;
+          }
+        default:
+          {
+            throw new IllegalStateException("Unsupported method: " + solrRequest.getMethod());
+          }
+      }
+      return processErrorsAndResponse(solrRequest, parserToUse, resp, url);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    } catch (HttpTimeoutException e) {
+      throw new SolrServerException(
+          "Timeout occurred while waiting response from server at: " + url, e);
+    } catch (SolrException se) {
+      throw se;
+    } catch (URISyntaxException | RuntimeException re) {
+      throw new SolrServerException(re);
+    } finally {
+      // See
+      // https://docs.oracle.com/en/java/javase/17/docs/api/java.net.http/java/net/http/HttpResponse.BodySubscribers.html#ofInputStream()
+      if (!wantStream(parserToUse)) {
+        try {
+          resp.body().close();
+        } catch (Exception e1) {
+          // ignore
+        }
+      }
+    }
+  }
+
+  private HttpResponse<InputStream> doGet(
+      String url,
+      HttpRequest.Builder reqb,
+      SolrRequest<?> solrRequest,
+      ModifiableSolrParams queryParams)
+      throws IOException, InterruptedException, URISyntaxException {
+    validateGetRequest(solrRequest);
+    reqb.GET();
+    decorateRequest(reqb, solrRequest);
+    reqb.uri(new URI(url + "?" + queryParams));
+    return httpClient.send(reqb.build(), HttpResponse.BodyHandlers.ofInputStream());
+  }
+
+  private HttpResponse<InputStream> doPutOrPost(
+      String url,
+      SolrRequest.METHOD method,
+      HttpRequest.Builder reqb,
+      SolrRequest<?> solrRequest,
+      ModifiableSolrParams queryParams)
+      throws IOException, InterruptedException, URISyntaxException {
+
+    final RequestWriter.ContentWriter contentWriter = requestWriter.getContentWriter(solrRequest);
+
+    final Collection<ContentStream> streams;
+    if (contentWriter == null) {
+      streams = requestWriter.getContentStreams(solrRequest);
+    } else {
+      streams = null;
+    }
+
+    String contentType = "application/x-www-form-urlencoded";
+    if (contentWriter != null && contentWriter.getContentType() != null) {
+      contentType = contentWriter.getContentType();
+    }
+    reqb.header("Content-Type", contentType);
+
+    if (isMultipart(streams)) {
+      throw new UnsupportedOperationException("This client does not support multipart.");
+    }
+
+    HttpRequest.BodyPublisher bodyPublisher;
+    Future<?> contentWritingFuture = null;
+    if (contentWriter != null) {
+      boolean success = maybeTryHeadRequest(url);
+      if (!success) {
+        reqb.version(HttpClient.Version.HTTP_1_1);
+      }
+
+      final PipedOutputStream source = new PipedOutputStream();
+      final PipedInputStream sink = new PipedInputStream(source);
+      bodyPublisher = HttpRequest.BodyPublishers.ofInputStream(() -> sink);
+
+      contentWritingFuture =
+          executor.submit(
+              () -> {
+                try (source) {
+                  contentWriter.write(source);
+                } catch (Exception e) {
+                  log.error("Cannot write Content Stream", e);
+                }
+              });
+    } else if (streams != null && streams.size() == 1) {
+      boolean success = maybeTryHeadRequest(url);
+      if (!success) {
+        reqb.version(HttpClient.Version.HTTP_1_1);
+      }
+
+      InputStream is = streams.iterator().next().getStream();
+      bodyPublisher = HttpRequest.BodyPublishers.ofInputStream(() -> is);
+    } else if (queryParams != null && urlParamNames != null) {
+      ModifiableSolrParams requestParams = queryParams;
+      queryParams = calculateQueryParams(urlParamNames, requestParams);
+      queryParams.add(calculateQueryParams(solrRequest.getQueryParams(), requestParams));
+      bodyPublisher = HttpRequest.BodyPublishers.ofString(requestParams.toString());
+    } else {
+      bodyPublisher = HttpRequest.BodyPublishers.noBody();
+    }
+
+    decorateRequest(reqb, solrRequest);
+    if (method == SolrRequest.METHOD.PUT) {
+      reqb.method("PUT", bodyPublisher);
+    } else {
+      reqb.method("POST", bodyPublisher);
+    }
+    URI uriWithQueryParams = new URI(url + "?" + queryParams);
+    reqb.uri(uriWithQueryParams);
+
+    HttpResponse<InputStream> response;
+    try {
+      response = httpClient.send(reqb.build(), HttpResponse.BodyHandlers.ofInputStream());
+    } finally {
+      if (contentWritingFuture != null) {
+        contentWritingFuture.cancel(true);
+      }
+    }
+    return response;
+  }
+
+  /**
+   * This is a workaround for the case where:
+   *
+   * <p>(1) no SSL/TLS (2) using POST with stream and (3) using Http/2
+   *
+   * <p>The JDK Http Client will send an upgrade request over Http/1 along with request content in
+   * the same request. However, the Jetty Server underpinning Solr does not accept this.
+   *
+   * <p>We send a HEAD request first, then the client knows if Solr can accept Http/2, and no
+   * additional upgrade requests will be sent.
+   *
+   * <p>See https://bugs.openjdk.org/browse/JDK-8287589 See
+   * https://github.com/jetty/jetty.project/issues/9998#issuecomment-1614216870
+   *
+   * <p>We only try once, and if it fails, we downgrade to Http/1
+   *
+   * @param url the url with no request parameters
+   * @return true if success
+   */
+  protected boolean maybeTryHeadRequest(String url) {
+    if (forceHttp11 || url == null || url.toLowerCase(Locale.ROOT).startsWith("https://")) {
+      return true;
+    }
+    return maybeTryHeadRequestSync(url);
+  }
+
+  protected volatile boolean headRequested; // must be threadsafe
+  private boolean headSucceeded; // must be threadsafe
+
+  private synchronized boolean maybeTryHeadRequestSync(String url) {
+    if (headRequested) {
+      return headSucceeded;
+    }
+
+    URI uriNoQueryParams;
+    try {
+      uriNoQueryParams = new URI(url);
+    } catch (URISyntaxException e) {
+      // If the url is invalid, let a subsequent request try again.
+      return false;
+    }
+    HttpRequest.Builder headReqB =
+        HttpRequest.newBuilder(uriNoQueryParams)
+            .method("HEAD", HttpRequest.BodyPublishers.noBody());
+    decorateRequest(headReqB, new QueryRequest());
+    try {
+      httpClient.send(headReqB.build(), HttpResponse.BodyHandlers.discarding());
+      headSucceeded = true;
+    } catch (IOException ioe) {
+      log.warn("Could not issue HEAD request to {} ", url, ioe);
+      headSucceeded = false;
+    } catch (InterruptedException ie) {
+      Thread.currentThread().interrupt();
+      headSucceeded = false;
+    } finally {
+
+      // The HEAD request is tried only once.  All future requests will skip this check.
+      headRequested = true;
+
+      if (!headSucceeded) {
+        log.info("All unencrypted POST requests with a chunked body will use http/1.1");
+      }
+    }
+
+    return headSucceeded;
+  }
+
+  private void decorateRequest(HttpRequest.Builder reqb, SolrRequest<?> solrRequest) {
+    if (requestTimeoutMillis > 0) {
+      reqb.timeout(Duration.of(requestTimeoutMillis, ChronoUnit.MILLIS));
+    } else {
+      reqb.timeout(Duration.of(idleTimeoutMillis, ChronoUnit.MILLIS));
+    }
+    reqb.header("User-Agent", USER_AGENT);
+    setBasicAuthHeader(solrRequest, reqb);
+    Map<String, String> headers = solrRequest.getHeaders();
+    if (headers != null) {
+      for (Map.Entry<String, String> entry : headers.entrySet()) {
+        reqb.header(entry.getKey(), entry.getValue());
+      }
+    }
+  }
+
+  private void setBasicAuthHeader(SolrRequest<?> solrRequest, HttpRequest.Builder reqb) {
+    if (solrRequest.getBasicAuthUser() != null && solrRequest.getBasicAuthPassword() != null) {
+      String encoded =
+          basicAuthCredentialsToAuthorizationString(
+              solrRequest.getBasicAuthUser(), solrRequest.getBasicAuthPassword());
+      reqb.header("Authorization", encoded);
+    } else if (basicAuthAuthorizationStr != null) {
+      reqb.header("Authorization", basicAuthAuthorizationStr);
+    }
+  }
+
+  private static final Pattern MIME_TYPE_PATTERN = Pattern.compile("[^;]*");
+
+  private String contentTypeToMimeType(String contentType) {
+    Matcher mimeTypeMatcher = MIME_TYPE_PATTERN.matcher(contentType);
+    return mimeTypeMatcher.find() ? mimeTypeMatcher.group() : null;
+  }
+
+  private static final Pattern CHARSET_PATTERN = Pattern.compile("(?i)^.*charset=(.*)$");
+
+  protected String contentTypeToEncoding(String contentType) {
+    Matcher encodingMatcher = CHARSET_PATTERN.matcher(contentType);
+    return encodingMatcher.find() ? encodingMatcher.group(1) : null;
+  }
+
+  private NamedList<Object> processErrorsAndResponse(
+      SolrRequest<?> solrRequest, ResponseParser parser, HttpResponse<InputStream> resp, String url)
+      throws SolrServerException {
+    String contentType = resp.headers().firstValue("Content-Type").orElse(null);
+    contentType = contentType == null ? "" : contentType;
+    String mimeType = contentTypeToMimeType(contentType);
+    String encoding = contentTypeToEncoding(contentType);
+    String method = resp.request() == null ? null : resp.request().method();
+    int status = resp.statusCode();
+    String reason = "" + status;
+    InputStream is = resp.body();
+    return processErrorsAndResponse(
+        status, reason, method, parser, is, mimeType, encoding, isV2ApiRequest(solrRequest), url);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (shutdownExecutor) {
+      ExecutorUtil.shutdownAndAwaitTermination(executor);
+    }
+    executor = null;
+
+    // TODO: Java 21 adds close/autoclosable to HttpClient.  We should use it.
+    httpClient = null;
+
+    assert ObjectReleaseTracker.release(this);
+  }
+
+  private void checkClosed() {
+    if (httpClient == null) {
+      throw new IllegalStateException("This is closed and cannot be reused.");
+    }
+  }
+
+  @Override
+  protected boolean isFollowRedirects() {
+    return httpClient.followRedirects() != HttpClient.Redirect.NEVER;
+  }
+
+  @Override
+  protected boolean processorAcceptsMimeType(
+      Collection<String> processorSupportedContentTypes, String mimeType) {
+    return processorSupportedContentTypes.stream()
+        .map(this::contentTypeToMimeType)
+        .filter(Objects::nonNull)
+        .map(String::trim)
+        .anyMatch(mimeType::equalsIgnoreCase);
+  }
+
+  @Override
+  protected void updateDefaultMimeTypeForParser() {
+    defaultParserMimeTypes =
+        parser.getContentTypes().stream()
+            .map(this::contentTypeToMimeType)
+            .filter(Objects::nonNull)
+            .map(s -> s.toLowerCase(Locale.ROOT).trim())
+            .collect(Collectors.toSet());
+  }
+
+  @Override
+  protected String allProcessorSupportedContentTypesCommaDelimited(
+      Collection<String> processorSupportedContentTypes) {
+    return processorSupportedContentTypes.stream()
+        .map(this::contentTypeToMimeType)
+        .filter(Objects::nonNull)
+        .map(s -> s.toLowerCase(Locale.ROOT).trim())
+        .collect(Collectors.joining(", "));
+  }
+
+  public static class Builder
+      extends HttpSolrClientBuilderBase<HttpJdkSolrClient.Builder, HttpJdkSolrClient> {
+
+    private SSLContext sslContext;
+
+    private CookieHandler cookieHandler;
+
+    public Builder() {
+      super();
+    }
+
+    public Builder(String baseSolrUrl) {
+      super();
+      this.baseSolrUrl = baseSolrUrl;
+    }
+
+    @Override
+    public HttpJdkSolrClient build() {
+      if (idleTimeoutMillis == null || idleTimeoutMillis <= 0) {
+        idleTimeoutMillis = (long) HttpClientUtil.DEFAULT_SO_TIMEOUT;
+      }
+      if (connectionTimeoutMillis == null) {
+        connectionTimeoutMillis = (long) HttpClientUtil.DEFAULT_CONNECT_TIMEOUT;
+      }
+      return new HttpJdkSolrClient(baseSolrUrl, this);
+    }
+
+    /**
+     * Use the provided SSLContext. See {@link
+     * java.net.http.HttpClient.Builder#sslContext(SSLContext)}.
+     *
+     * @param sslContext the ssl context to use
+     * @return this Builder
+     */
+    public HttpJdkSolrClient.Builder withSSLContext(SSLContext sslContext) {
+      this.sslContext = sslContext;
+      return this;
+    }
+
+    /**
+     * Use the provided CookieHandler.
+     *
+     * @param cookieHandler the cookie handler to use
+     * @return this Builder
+     */
+    public HttpJdkSolrClient.Builder withCookieHandler(CookieHandler cookieHandler) {
+      this.cookieHandler = cookieHandler;
+      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 180135d8273..805bbd5f6f9 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
@@ -95,7 +95,7 @@ import org.slf4j.MDC;
 /**
  * A SolrClient implementation that talks directly to a Solr server via Apache HTTP client
  *
- * @deprecated Please use {@link Http2SolrClient}
+ * @deprecated Please use {@link Http2SolrClient} or {@link HttpJdkSolrClient}
  */
 @Deprecated(since = "9.0")
 public class HttpSolrClient extends BaseHttpSolrClient {
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClientBase.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClientBase.java
new file mode 100644
index 00000000000..c1171af454f
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClientBase.java
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.impl;
+
+import static org.apache.solr.common.util.Utils.getObjectByPath;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.solr.client.solrj.ResponseParser;
+import org.apache.solr.client.solrj.SolrClient;
+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.common.SolrException;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Utils;
+
+public abstract class HttpSolrClientBase extends SolrClient {
+
+  protected static final String DEFAULT_PATH = "/select";
+  protected static final Charset FALLBACK_CHARSET = StandardCharsets.UTF_8;
+  private static final List<String> errPath = Arrays.asList("metadata", "error-class");
+
+  /** The URL of the Solr server. */
+  protected final String serverBaseUrl;
+
+  protected final long idleTimeoutMillis;
+
+  protected final long requestTimeoutMillis;
+
+  protected final Set<String> urlParamNames;
+
+  protected RequestWriter requestWriter = new BinaryRequestWriter();
+
+  // updating parser instance needs to go via the setter to ensure update of defaultParserMimeTypes
+  protected ResponseParser parser = new BinaryResponseParser();
+
+  protected Set<String> defaultParserMimeTypes;
+
+  protected final String basicAuthAuthorizationStr;
+
+  protected HttpSolrClientBase(String serverBaseUrl, HttpSolrClientBuilderBase<?, ?> builder) {
+    if (serverBaseUrl != null) {
+      if (!serverBaseUrl.equals("/") && serverBaseUrl.endsWith("/")) {
+        serverBaseUrl = serverBaseUrl.substring(0, serverBaseUrl.length() - 1);
+      }
+
+      if (serverBaseUrl.startsWith("//")) {
+        serverBaseUrl = serverBaseUrl.substring(1, serverBaseUrl.length());
+      }
+      this.serverBaseUrl = serverBaseUrl;
+    } else {
+      this.serverBaseUrl = null;
+    }
+    this.idleTimeoutMillis = builder.idleTimeoutMillis;
+    this.basicAuthAuthorizationStr = builder.basicAuthAuthorizationStr;
+    if (builder.requestWriter != null) {
+      this.requestWriter = builder.requestWriter;
+    }
+    if (builder.responseParser != null) {
+      this.parser = builder.responseParser;
+    }
+    this.defaultCollection = builder.defaultCollection;
+    if (builder.requestTimeoutMillis != null) {
+      this.requestTimeoutMillis = builder.requestTimeoutMillis;
+    } else {
+      this.requestTimeoutMillis = -1;
+    }
+    if (builder.urlParamNames != null) {
+      this.urlParamNames = builder.urlParamNames;
+    } else {
+      this.urlParamNames = Set.of();
+    }
+  }
+
+  protected String getRequestPath(SolrRequest<?> solrRequest, String collection)
+      throws MalformedURLException {
+    String basePath = solrRequest.getBasePath() == null ? serverBaseUrl : solrRequest.getBasePath();
+    if (collection != null) basePath += "/" + collection;
+
+    if (solrRequest instanceof V2Request) {
+      if (System.getProperty("solr.v2RealPath") == null) {
+        basePath = changeV2RequestEndpoint(basePath);
+      } else {
+        basePath = serverBaseUrl + "/____v2";
+      }
+    }
+    String path = requestWriter.getPath(solrRequest);
+    if (path == null || !path.startsWith("/")) {
+      path = DEFAULT_PATH;
+    }
+
+    return basePath + path;
+  }
+
+  protected String changeV2RequestEndpoint(String basePath) throws MalformedURLException {
+    URL oldURL = new URL(basePath);
+    String newPath = oldURL.getPath().replaceFirst("/solr", "/api");
+    return new URL(oldURL.getProtocol(), oldURL.getHost(), oldURL.getPort(), newPath).toString();
+  }
+
+  protected ResponseParser responseParser(SolrRequest<?> solrRequest) {
+    // TODO add invariantParams support
+    return solrRequest.getResponseParser() == null ? this.parser : solrRequest.getResponseParser();
+  }
+
+  protected ModifiableSolrParams initalizeSolrParams(
+      SolrRequest<?> solrRequest, ResponseParser parserToUse) {
+    // The parser 'wt=' and 'version=' params are used instead of the original
+    // params
+    ModifiableSolrParams wparams = new ModifiableSolrParams(solrRequest.getParams());
+    wparams.set(CommonParams.WT, parserToUse.getWriterType());
+    wparams.set(CommonParams.VERSION, parserToUse.getVersion());
+    return wparams;
+  }
+
+  protected boolean isMultipart(Collection<ContentStream> streams) {
+    boolean isMultipart = false;
+    if (streams != null) {
+      boolean hasNullStreamName = false;
+      hasNullStreamName = streams.stream().anyMatch(cs -> cs.getName() == null);
+      isMultipart = !hasNullStreamName && streams.size() > 1;
+    }
+    return isMultipart;
+  }
+
+  protected ModifiableSolrParams calculateQueryParams(
+      Set<String> queryParamNames, ModifiableSolrParams wparams) {
+    ModifiableSolrParams queryModParams = new ModifiableSolrParams();
+    if (queryParamNames != null) {
+      for (String param : queryParamNames) {
+        String[] value = wparams.getParams(param);
+        if (value != null) {
+          for (String v : value) {
+            queryModParams.add(param, v);
+          }
+          wparams.remove(param);
+        }
+      }
+    }
+    return queryModParams;
+  }
+
+  protected void validateGetRequest(SolrRequest<?> solrRequest) throws IOException {
+    RequestWriter.ContentWriter contentWriter = requestWriter.getContentWriter(solrRequest);
+    Collection<ContentStream> streams =
+        contentWriter == null ? requestWriter.getContentStreams(solrRequest) : null;
+    if (contentWriter != null || streams != null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "GET can't send streams!");
+    }
+  }
+
+  protected abstract boolean isFollowRedirects();
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  protected NamedList<Object> processErrorsAndResponse(
+      int httpStatus,
+      String responseReason,
+      String responseMethod,
+      final ResponseParser processor,
+      InputStream is,
+      String mimeType,
+      String encoding,
+      final boolean isV2Api,
+      String urlExceptionMessage)
+      throws SolrServerException {
+    boolean shouldClose = true;
+    try {
+      // handle some http level checks before trying to parse the response
+      switch (httpStatus) {
+        case 200: // OK
+        case 400: // Bad Request
+        case 409: // Conflict
+          break;
+        case 301: // Moved Permanently
+        case 302: // Moved Temporarily
+          if (!isFollowRedirects()) {
+            throw new SolrServerException(
+                "Server at " + urlExceptionMessage + " sent back a redirect (" + httpStatus + ").");
+          }
+          break;
+        default:
+          if (processor == null || mimeType == null) {
+            throw new BaseHttpSolrClient.RemoteSolrException(
+                urlExceptionMessage,
+                httpStatus,
+                "non ok status: " + httpStatus + ", message:" + responseReason,
+                null);
+          }
+      }
+
+      if (wantStream(processor)) {
+        // no processor specified, return raw stream
+        NamedList<Object> rsp = new NamedList<>();
+        rsp.add("stream", is);
+        rsp.add("responseStatus", httpStatus);
+        // Only case where stream should not be closed
+        shouldClose = false;
+        return rsp;
+      }
+
+      checkContentType(processor, is, mimeType, encoding, httpStatus, urlExceptionMessage);
+
+      NamedList<Object> rsp;
+      try {
+        rsp = processor.processResponse(is, encoding);
+      } catch (Exception e) {
+        throw new BaseHttpSolrClient.RemoteSolrException(
+            urlExceptionMessage, httpStatus, e.getMessage(), e);
+      }
+
+      Object error = rsp == null ? null : rsp.get("error");
+      if (rsp != null && error == null && processor instanceof NoOpResponseParser) {
+        error = rsp.get("response");
+      }
+      if (error != null
+          && (String.valueOf(getObjectByPath(error, true, errPath))
+              .endsWith("ExceptionWithErrObject"))) {
+        throw BaseHttpSolrClient.RemoteExecutionException.create(urlExceptionMessage, rsp);
+      }
+      if (httpStatus != 200 && !isV2Api) {
+        NamedList<String> metadata = null;
+        String reason = null;
+        try {
+          if (error != null) {
+            reason = (String) Utils.getObjectByPath(error, false, Collections.singletonList("msg"));
+            if (reason == null) {
+              reason =
+                  (String) Utils.getObjectByPath(error, false, Collections.singletonList("trace"));
+            }
+            Object metadataObj =
+                Utils.getObjectByPath(error, false, Collections.singletonList("metadata"));
+            if (metadataObj instanceof NamedList) {
+              metadata = (NamedList<String>) metadataObj;
+            } else if (metadataObj instanceof List) {
+              // NamedList parsed as List convert to NamedList again
+              List<Object> list = (List<Object>) metadataObj;
+              metadata = new NamedList<>(list.size() / 2);
+              for (int i = 0; i < list.size(); i += 2) {
+                metadata.add((String) list.get(i), (String) list.get(i + 1));
+              }
+            } else if (metadataObj instanceof Map) {
+              metadata = new NamedList((Map) metadataObj);
+            }
+          }
+        } catch (Exception ex) {
+          /* Ignored */
+        }
+        if (reason == null) {
+          StringBuilder msg = new StringBuilder();
+          msg.append(responseReason).append("\n").append("request: ").append(responseMethod);
+          if (error != null) {
+            msg.append("\n\nError returned:\n").append(error);
+          }
+          reason = java.net.URLDecoder.decode(msg.toString(), FALLBACK_CHARSET);
+        }
+        BaseHttpSolrClient.RemoteSolrException rss =
+            new BaseHttpSolrClient.RemoteSolrException(
+                urlExceptionMessage, httpStatus, reason, null);
+        if (metadata != null) rss.setMetadata(metadata);
+        throw rss;
+      }
+      return rsp;
+    } finally {
+      if (shouldClose) {
+        try {
+          is.close();
+        } catch (IOException e) {
+          // quitely
+        }
+      }
+    }
+  }
+
+  protected boolean wantStream(final ResponseParser processor) {
+    return processor == null || processor instanceof InputStreamResponseParser;
+  }
+
+  protected abstract boolean processorAcceptsMimeType(
+      Collection<String> processorSupportedContentTypes, String mimeType);
+
+  protected abstract String allProcessorSupportedContentTypesCommaDelimited(
+      Collection<String> processorSupportedContentTypes);
+
+  /**
+   * Validates that the content type in the response can be processed by the Response Parser. Throws
+   * a {@code RemoteSolrException} if not.
+   */
+  private void checkContentType(
+      ResponseParser processor,
+      InputStream is,
+      String mimeType,
+      String encoding,
+      int httpStatus,
+      String urlExceptionMessage) {
+    if (mimeType == null
+        || (processor == this.parser && defaultParserMimeTypes.contains(mimeType))) {
+      // Shortcut the default scenario
+      return;
+    }
+    final Collection<String> processorSupportedContentTypes = processor.getContentTypes();
+    if (processorSupportedContentTypes != null && !processorSupportedContentTypes.isEmpty()) {
+      boolean processorAcceptsMimeType =
+          processorAcceptsMimeType(processorSupportedContentTypes, mimeType);
+      if (!processorAcceptsMimeType) {
+        // unexpected mime type
+        final String allSupportedTypes =
+            allProcessorSupportedContentTypesCommaDelimited(processorSupportedContentTypes);
+        String prefix =
+            "Expected mime type in [" + allSupportedTypes + "] but got " + mimeType + ". ";
+        String exceptionEncoding = encoding != null ? encoding : FALLBACK_CHARSET.name();
+        try {
+          ByteArrayOutputStream body = new ByteArrayOutputStream();
+          is.transferTo(body);
+          throw new BaseHttpSolrClient.RemoteSolrException(
+              urlExceptionMessage, httpStatus, prefix + body.toString(exceptionEncoding), null);
+        } catch (IOException e) {
+          throw new BaseHttpSolrClient.RemoteSolrException(
+              urlExceptionMessage,
+              httpStatus,
+              "Could not parse response with encoding " + exceptionEncoding,
+              e);
+        }
+      }
+    }
+  }
+
+  protected static String basicAuthCredentialsToAuthorizationString(String user, String pass) {
+    String userPass = user + ":" + pass;
+    return "Basic " + Base64.getEncoder().encodeToString(userPass.getBytes(FALLBACK_CHARSET));
+  }
+
+  protected void setParser(ResponseParser parser) {
+    this.parser = parser;
+    updateDefaultMimeTypeForParser();
+  }
+
+  protected abstract void updateDefaultMimeTypeForParser();
+
+  public boolean isV2ApiRequest(final SolrRequest<?> request) {
+    return request instanceof V2Request || request.getPath().contains("/____v2");
+  }
+
+  public String getBaseURL() {
+    return serverBaseUrl;
+  }
+
+  public ResponseParser getParser() {
+    return parser;
+  }
+
+  public long getIdleTimeout() {
+    return idleTimeoutMillis;
+  }
+
+  public Set<String> getUrlParamNames() {
+    return urlParamNames;
+  }
+}
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClientBuilderBase.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClientBuilderBase.java
new file mode 100644
index 00000000000..19025c9b6ec
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClientBuilderBase.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.impl;
+
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.solr.client.solrj.ResponseParser;
+import org.apache.solr.client.solrj.request.RequestWriter;
+
+public abstract class HttpSolrClientBuilderBase<
+    B extends HttpSolrClientBuilderBase<?, ?>, C extends HttpSolrClientBase> {
+  protected Long idleTimeoutMillis;
+  protected Long connectionTimeoutMillis;
+  protected Long requestTimeoutMillis;
+  protected String basicAuthAuthorizationStr;
+  protected Boolean followRedirects;
+  protected String baseSolrUrl;
+  protected RequestWriter requestWriter;
+  protected ResponseParser responseParser;
+  protected String defaultCollection;
+  protected Set<String> urlParamNames;
+  protected Integer maxConnectionsPerHost;
+  protected ExecutorService executor;
+  protected boolean useHttp1_1 = Boolean.getBoolean("solr.http1");
+  protected String proxyHost;
+  protected int proxyPort;
+  protected boolean proxyIsSocks4;
+  protected boolean proxyIsSecure;
+
+  public abstract C build();
+
+  /** Provides a {@link RequestWriter} for created clients to use when handing requests. */
+  @SuppressWarnings("unchecked")
+  public B withRequestWriter(RequestWriter requestWriter) {
+    this.requestWriter = requestWriter;
+    return (B) this;
+  }
+
+  /** Provides a {@link ResponseParser} for created clients to use when handling requests. */
+  @SuppressWarnings("unchecked")
+  public B withResponseParser(ResponseParser responseParser) {
+    this.responseParser = responseParser;
+    return (B) this;
+  }
+
+  /** Sets a default for core or collection based requests. */
+  @SuppressWarnings("unchecked")
+  public B withDefaultCollection(String defaultCoreOrCollection) {
+    this.defaultCollection = defaultCoreOrCollection;
+    return (B) this;
+  }
+
+  @SuppressWarnings("unchecked")
+  public B withFollowRedirects(boolean followRedirects) {
+    this.followRedirects = followRedirects;
+    return (B) this;
+  }
+
+  @SuppressWarnings("unchecked")
+  public B withExecutor(ExecutorService executor) {
+    this.executor = executor;
+    return (B) this;
+  }
+
+  @SuppressWarnings("unchecked")
+  public B withBasicAuthCredentials(String user, String pass) {
+    if (user != null || pass != null) {
+      if (user == null || pass == null) {
+        throw new IllegalStateException(
+            "Invalid Authentication credentials. Either both username and password or none must be provided");
+      }
+    }
+    this.basicAuthAuthorizationStr =
+        Http2SolrClient.basicAuthCredentialsToAuthorizationString(user, pass);
+    return (B) this;
+  }
+
+  /**
+   * Expert Method
+   *
+   * @param urlParamNames set of param keys that are only sent via the query string. Note that the
+   *     param will be sent as a query string if the key is part of this Set or the SolrRequest's
+   *     query params.
+   * @see org.apache.solr.client.solrj.SolrRequest#getQueryParams
+   */
+  @SuppressWarnings("unchecked")
+  public B withTheseParamNamesInTheUrl(Set<String> urlParamNames) {
+    this.urlParamNames = urlParamNames;
+    return (B) this;
+  }
+
+  /**
+   * Set maxConnectionsPerHost for http1 connections, maximum number http2 connections is limited to
+   * 4
+   */
+  @SuppressWarnings("unchecked")
+  public B withMaxConnectionsPerHost(int max) {
+    this.maxConnectionsPerHost = max;
+    return (B) this;
+  }
+
+  @SuppressWarnings("unchecked")
+  public B withIdleTimeout(long idleConnectionTimeout, TimeUnit unit) {
+    this.idleTimeoutMillis = TimeUnit.MILLISECONDS.convert(idleConnectionTimeout, unit);
+    return (B) this;
+  }
+
+  public Long getIdleTimeoutMillis() {
+    return idleTimeoutMillis;
+  }
+
+  @SuppressWarnings("unchecked")
+  public B withConnectionTimeout(long connectionTimeout, TimeUnit unit) {
+    this.connectionTimeoutMillis = TimeUnit.MILLISECONDS.convert(connectionTimeout, unit);
+    return (B) this;
+  }
+
+  public Long getConnectionTimeout() {
+    return connectionTimeoutMillis;
+  }
+
+  /**
+   * Set a timeout in milliseconds for requests issued by this client.
+   *
+   * @param requestTimeout The timeout in milliseconds
+   * @return this Builder.
+   */
+  @SuppressWarnings("unchecked")
+  public B withRequestTimeout(long requestTimeout, TimeUnit unit) {
+    this.requestTimeoutMillis = TimeUnit.MILLISECONDS.convert(requestTimeout, unit);
+    return (B) this;
+  }
+
+  /**
+   * If true, prefer http1.1 over http2. If not set, the default is determined by system property
+   * 'solr.http1'. Otherwise, false.
+   *
+   * @param useHttp1_1 prefer http1.1?
+   * @return this Builder
+   */
+  @SuppressWarnings("unchecked")
+  public B useHttp1_1(boolean useHttp1_1) {
+    this.useHttp1_1 = useHttp1_1;
+    return (B) this;
+  }
+
+  /**
+   * Setup a proxy
+   *
+   * @param host The proxy host
+   * @param port The proxy port
+   * @param isSocks4 If true creates an SOCKS 4 proxy, otherwise creates an HTTP proxy
+   * @param isSecure If true enables the secure flag on the proxy
+   * @return this Builder
+   */
+  @SuppressWarnings("unchecked")
+  public B withProxyConfiguration(String host, int port, boolean isSocks4, boolean isSecure) {
+    this.proxyHost = host;
+    this.proxyPort = port;
+    this.proxyIsSocks4 = isSocks4;
+    this.proxyIsSecure = isSecure;
+    return (B) this;
+  }
+
+  /**
+   * Setup basic authentication from a string formatted as username:password. If the string is Null
+   * then it doesn't do anything.
+   *
+   * @param credentials The username and password formatted as username:password
+   * @return this Builder
+   */
+  @SuppressWarnings("unchecked")
+  public B withOptionalBasicAuthCredentials(String credentials) {
+    if (credentials != null) {
+      if (credentials.indexOf(':') == -1) {
+        throw new IllegalStateException(
+            "Invalid Authentication credential formatting. Provide username and password in the 'username:password' format.");
+      }
+      String username = credentials.substring(0, credentials.indexOf(':'));
+      String password = credentials.substring(credentials.indexOf(':') + 1, credentials.length());
+      withBasicAuthCredentials(username, password);
+    }
+    return (B) this;
+  }
+}
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/DebugServlet.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/DebugServlet.java
new file mode 100644
index 00000000000..272d9223dcf
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/DebugServlet.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.solr.common.util.SuppressForbidden;
+
+public class DebugServlet extends HttpServlet {
+  public static void clear() {
+    lastMethod = null;
+    headers = null;
+    parameters = null;
+    errorCode = null;
+    queryString = null;
+    cookies = null;
+    responseHeaders = null;
+    responseBody = null;
+  }
+
+  public static Integer errorCode = null;
+  public static String lastMethod = null;
+  public static HashMap<String, String> headers = null;
+  public static Map<String, String[]> parameters = null;
+  public static String queryString = null;
+  public static javax.servlet.http.Cookie[] cookies = null;
+  public static List<String[]> responseHeaders = null;
+  public static Object responseBody = null;
+  public static byte[] requestBody = null;
+
+  public static void setErrorCode(Integer code) {
+    errorCode = code;
+  }
+
+  public static void addResponseHeader(String headerName, String headerValue) {
+    if (responseHeaders == null) {
+      responseHeaders = new ArrayList<>();
+    }
+    responseHeaders.add(new String[] {headerName, headerValue});
+  }
+
+  @Override
+  protected void doDelete(HttpServletRequest req, HttpServletResponse resp)
+      throws ServletException, IOException {
+    lastMethod = "delete";
+    recordRequest(req, resp);
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest req, HttpServletResponse resp)
+      throws ServletException, IOException {
+    lastMethod = "get";
+    recordRequest(req, resp);
+  }
+
+  @Override
+  protected void doHead(HttpServletRequest req, HttpServletResponse resp)
+      throws ServletException, IOException {
+    lastMethod = "head";
+    recordRequest(req, resp);
+  }
+
+  private void setHeaders(HttpServletRequest req) {
+    Enumeration<String> headerNames = req.getHeaderNames();
+    headers = new HashMap<>();
+    while (headerNames.hasMoreElements()) {
+      final String name = headerNames.nextElement();
+      headers.put(name.toLowerCase(Locale.getDefault()), req.getHeader(name));
+    }
+  }
+
+  @SuppressForbidden(reason = "fake servlet only")
+  private void setParameters(HttpServletRequest req) {
+    parameters = req.getParameterMap();
+  }
+
+  private void setQueryString(HttpServletRequest req) {
+    queryString = req.getQueryString();
+  }
+
+  private void setCookies(HttpServletRequest req) {
+    javax.servlet.http.Cookie[] ck = req.getCookies();
+    cookies = req.getCookies();
+  }
+
+  @Override
+  protected void doPost(HttpServletRequest req, HttpServletResponse resp)
+      throws ServletException, IOException {
+    lastMethod = "post";
+    recordRequest(req, resp);
+  }
+
+  @Override
+  protected void doPut(HttpServletRequest req, HttpServletResponse resp)
+      throws ServletException, IOException {
+    lastMethod = "put";
+    recordRequest(req, resp);
+  }
+
+  private void recordRequest(HttpServletRequest req, HttpServletResponse resp) {
+    setHeaders(req);
+    setParameters(req);
+    setQueryString(req);
+    setCookies(req);
+    try {
+      requestBody = req.getInputStream().readAllBytes();
+    } catch (Exception e) {
+      // ignore
+    }
+    if (responseHeaders != null) {
+      for (String[] h : responseHeaders) {
+        resp.addHeader(h[0], h[1]);
+      }
+    }
+    if (responseBody != null) {
+      try {
+        if (responseBody instanceof String) {
+          resp.getWriter().print((String) responseBody);
+        } else if (responseBody instanceof byte[]) {
+          resp.getOutputStream().write((byte[]) responseBody);
+        } else {
+          throw new IllegalArgumentException(
+              "Only String and byte[] are supported for responseBody.");
+        }
+      } catch (IOException ioe) {
+        throw new RuntimeException(ioe);
+      }
+    }
+    if (null != errorCode) {
+      try {
+        resp.sendError(errorCode);
+      } catch (IOException e) {
+        throw new RuntimeException("sendError IO fail in DebugServlet", e);
+      }
+    }
+  }
+}
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientCompatibilityTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientCompatibilityTest.java
index d881463cd10..835f4c04cbe 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientCompatibilityTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientCompatibilityTest.java
@@ -47,7 +47,7 @@ public class Http2SolrClientCompatibilityTest extends SolrJettyTestBase {
 
     JettyConfig jettyConfig =
         JettyConfig.builder()
-            .withServlet(new ServletHolder(Http2SolrClientTest.DebugServlet.class), "/debug/*")
+            .withServlet(new ServletHolder(DebugServlet.class), "/debug/*")
             .useOnlyHttp1(true)
             .build();
     createAndStartJetty(legacyExampleCollection1SolrHome(), jettyConfig);
@@ -68,7 +68,7 @@ public class Http2SolrClientCompatibilityTest extends SolrJettyTestBase {
 
     JettyConfig jettyConfig =
         JettyConfig.builder()
-            .withServlet(new ServletHolder(Http2SolrClientTest.DebugServlet.class), "/debug/*")
+            .withServlet(new ServletHolder(DebugServlet.class), "/debug/*")
             .useOnlyHttp1(false)
             .build();
     createAndStartJetty(legacyExampleCollection1SolrHome(), jettyConfig);
@@ -91,7 +91,7 @@ public class Http2SolrClientCompatibilityTest extends SolrJettyTestBase {
     // then notify this to users
     JettyConfig jettyConfig =
         JettyConfig.builder()
-            .withServlet(new ServletHolder(Http2SolrClientTest.DebugServlet.class), "/debug/*")
+            .withServlet(new ServletHolder(DebugServlet.class), "/debug/*")
             .useOnlyHttp1(true)
             .build();
     createAndStartJetty(legacyExampleCollection1SolrHome(), jettyConfig);
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientProxyTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientProxyTest.java
index 7333ecfbeec..fd9ef539c02 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientProxyTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientProxyTest.java
@@ -20,10 +20,12 @@ import com.carrotsearch.randomizedtesting.RandomizedTest;
 import java.util.Properties;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.cloud.SocketProxy;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.embedded.JettyConfig;
 import org.apache.solr.util.ExternalPaths;
 import org.apache.solr.util.SolrJettyTestRule;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -47,31 +49,66 @@ public class Http2SolrClientProxyTest extends SolrTestCaseJ4 {
         .create();
   }
 
-  /** Setup a simple http proxy and verify a request works */
-  @Test
-  public void testProxy() throws Exception {
-    var proxy = solrClientTestRule.getJetty().getProxy();
-    assertNotNull(proxy);
+  private SocketProxy proxy;
+
+  private String host;
+
+  private String url;
 
-    String host = proxy.getUrl().getHost();
-    String url = "http://" + host + ":" + (proxy.getUrl().getPort() + 10) + "/solr";
+  @Before
+  public void before() {
+    this.proxy = solrClientTestRule.getJetty().getProxy();
+    this.host = proxy.getUrl().getHost();
+    this.url = "http://" + host + ":" + (proxy.getUrl().getPort() + 10) + "/solr";
+  }
 
+  @Test
+  public void testProxyWithHttp2SolrClient() throws Exception {
+    assertNotNull(proxy);
     var builder =
         new Http2SolrClient.Builder(url)
             .withProxyConfiguration(host, proxy.getListenPort(), false, false);
 
     try (Http2SolrClient client = builder.build()) {
-      String id = "1234";
-      SolrInputDocument doc = new SolrInputDocument();
-      doc.addField("id", id);
-      client.add(DEFAULT_TEST_COLLECTION_NAME, doc);
-      client.commit(DEFAULT_TEST_COLLECTION_NAME);
-      assertEquals(
-          1,
-          client
-              .query(DEFAULT_TEST_COLLECTION_NAME, new SolrQuery("id:" + id))
-              .getResults()
-              .getNumFound());
+      testProxy(client);
     }
   }
+
+  @Test
+  public void testProxyWithHttpSolrClientJdkImpl() throws Exception {
+    assertNotNull(proxy);
+    var builder =
+        new HttpJdkSolrClient.Builder(url)
+            .withProxyConfiguration(host, proxy.getListenPort(), false, false);
+    try (HttpJdkSolrClient client = builder.build()) {
+      testProxy(client);
+    }
+    // This is a workaround for java.net.http.HttpClient not implementing closeable/autoclosable
+    // until Java 21.
+    System.gc();
+  }
+
+  /** Setup a simple http proxy and verify a request works */
+  public void testProxy(HttpSolrClientBase client) throws Exception {
+    String id = "1234";
+    SolrInputDocument doc = new SolrInputDocument();
+    doc.addField("id", id);
+    client.add(DEFAULT_TEST_COLLECTION_NAME, doc);
+    client.commit(DEFAULT_TEST_COLLECTION_NAME);
+    assertEquals(
+        1,
+        client
+            .query(DEFAULT_TEST_COLLECTION_NAME, new SolrQuery("id:" + id))
+            .getResults()
+            .getNumFound());
+
+    client.deleteByQuery(DEFAULT_TEST_COLLECTION_NAME, "*:*");
+    client.commit(DEFAULT_TEST_COLLECTION_NAME);
+    assertEquals(
+        0,
+        client
+            .query(DEFAULT_TEST_COLLECTION_NAME, new SolrQuery("*:*"))
+            .getResults()
+            .getNumFound());
+  }
 }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientTest.java
index 9b57e7aa2c5..5b4f7eb86af 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientTest.java
@@ -17,208 +17,51 @@
 
 package org.apache.solr.client.solrj.impl;
 
-import static org.hamcrest.CoreMatchers.instanceOf;
-
 import java.io.IOException;
-import java.io.InputStream;
 import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
 import java.util.Base64;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Enumeration;
 import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import org.apache.solr.SolrJettyTestBase;
+import org.apache.solr.client.solrj.ResponseParser;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.request.RequestWriter;
-import org.apache.solr.client.solrj.request.SolrPing;
-import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
-import org.apache.solr.common.util.SuppressForbidden;
-import org.apache.solr.embedded.JettyConfig;
 import org.eclipse.jetty.client.WWWAuthenticationProtocolHandler;
 import org.eclipse.jetty.http.HttpStatus;
-import org.eclipse.jetty.servlet.ServletHolder;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
-import org.hamcrest.MatcherAssert;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class Http2SolrClientTest extends SolrJettyTestBase {
-
-  private static final String EXPECTED_USER_AGENT =
-      "Solr[" + Http2SolrClient.class.getName() + "] 2.0";
-
-  private static final String DEFAULT_CORE = "foo";
-  private static final String SLOW_SERVLET_PATH = "/slow";
-  private static final String SLOW_SERVLET_REGEX = SLOW_SERVLET_PATH + "/*";
-  private static final String DEBUG_SERVLET_PATH = "/debug";
-  private static final String DEBUG_SERVLET_REGEX = DEBUG_SERVLET_PATH + "/*";
-  private static final String REDIRECT_SERVLET_PATH = "/redirect";
-  private static final String REDIRECT_SERVLET_REGEX = REDIRECT_SERVLET_PATH + "/*";
-
-  public static class DebugServlet extends HttpServlet {
-    public static void clear() {
-      lastMethod = null;
-      headers = null;
-      parameters = null;
-      errorCode = null;
-      queryString = null;
-      cookies = null;
-      responseHeaders = null;
-    }
-
-    public static Integer errorCode = null;
-    public static String lastMethod = null;
-    public static HashMap<String, String> headers = null;
-    public static Map<String, String[]> parameters = null;
-    public static String queryString = null;
-    public static javax.servlet.http.Cookie[] cookies = null;
-    public static List<String[]> responseHeaders = null;
-
-    public static void setErrorCode(Integer code) {
-      errorCode = code;
-    }
-
-    public static void addResponseHeader(String headerName, String headerValue) {
-      if (responseHeaders == null) {
-        responseHeaders = new ArrayList<>();
-      }
-      responseHeaders.add(new String[] {headerName, headerValue});
-    }
-
-    @Override
-    protected void doDelete(HttpServletRequest req, HttpServletResponse resp)
-        throws ServletException, IOException {
-      lastMethod = "delete";
-      recordRequest(req, resp);
-    }
-
-    @Override
-    protected void doGet(HttpServletRequest req, HttpServletResponse resp)
-        throws ServletException, IOException {
-      lastMethod = "get";
-      recordRequest(req, resp);
-    }
-
-    @Override
-    protected void doHead(HttpServletRequest req, HttpServletResponse resp)
-        throws ServletException, IOException {
-      lastMethod = "head";
-      recordRequest(req, resp);
-    }
-
-    private void setHeaders(HttpServletRequest req) {
-      Enumeration<String> headerNames = req.getHeaderNames();
-      headers = new HashMap<>();
-      while (headerNames.hasMoreElements()) {
-        final String name = headerNames.nextElement();
-        headers.put(name.toLowerCase(Locale.getDefault()), req.getHeader(name));
-      }
-    }
-
-    @SuppressForbidden(reason = "fake servlet only")
-    private void setParameters(HttpServletRequest req) {
-      parameters = req.getParameterMap();
-    }
-
-    private void setQueryString(HttpServletRequest req) {
-      queryString = req.getQueryString();
-    }
-
-    private void setCookies(HttpServletRequest req) {
-      javax.servlet.http.Cookie[] ck = req.getCookies();
-      cookies = req.getCookies();
-    }
-
-    @Override
-    protected void doPost(HttpServletRequest req, HttpServletResponse resp)
-        throws ServletException, IOException {
-      lastMethod = "post";
-      recordRequest(req, resp);
-    }
-
-    @Override
-    protected void doPut(HttpServletRequest req, HttpServletResponse resp)
-        throws ServletException, IOException {
-      lastMethod = "put";
-      recordRequest(req, resp);
-    }
-
-    private void recordRequest(HttpServletRequest req, HttpServletResponse resp) {
-      setHeaders(req);
-      setParameters(req);
-      setQueryString(req);
-      setCookies(req);
-      if (responseHeaders != null) {
-        for (String[] h : responseHeaders) {
-          resp.addHeader(h[0], h[1]);
-        }
-      }
-      if (null != errorCode) {
-        try {
-          resp.sendError(errorCode);
-        } catch (IOException e) {
-          throw new RuntimeException("sendError IO fail in DebugServlet", e);
-        }
-      }
-    }
-  }
-
-  @BeforeClass
-  public static void beforeTest() throws Exception {
-    JettyConfig jettyConfig =
-        JettyConfig.builder()
-            .withServlet(
-                new ServletHolder(BasicHttpSolrClientTest.RedirectServlet.class),
-                REDIRECT_SERVLET_REGEX)
-            .withServlet(
-                new ServletHolder(BasicHttpSolrClientTest.SlowServlet.class), SLOW_SERVLET_REGEX)
-            .withServlet(new ServletHolder(DebugServlet.class), DEBUG_SERVLET_REGEX)
-            .build();
-    createAndStartJetty(legacyExampleCollection1SolrHome(), jettyConfig);
-  }
+public class Http2SolrClientTest extends HttpSolrClientTestBase {
 
   @Override
-  public void tearDown() throws Exception {
-    System.clearProperty("basicauth");
-    System.clearProperty(HttpClientUtil.SYS_PROP_HTTP_CLIENT_BUILDER_FACTORY);
-    DebugServlet.clear();
-    super.tearDown();
+  protected String expectedUserAgent() {
+    return "Solr[" + Http2SolrClient.class.getName() + "] 2.0";
   }
 
-  private Http2SolrClient.Builder getHttp2SolrClientBuilder(
+  @Override
+  @SuppressWarnings(value = "unchecked")
+  protected <B extends HttpSolrClientBuilderBase<?, ?>> B builder(
       String url, int connectionTimeout, int socketTimeout) {
-    return new Http2SolrClient.Builder(url)
-        .withConnectionTimeout(connectionTimeout, TimeUnit.MILLISECONDS)
-        .withIdleTimeout(socketTimeout, TimeUnit.MILLISECONDS);
+    Http2SolrClient.Builder b =
+        new Http2SolrClient.Builder(url)
+            .withConnectionTimeout(connectionTimeout, TimeUnit.MILLISECONDS)
+            .withIdleTimeout(socketTimeout, TimeUnit.MILLISECONDS);
+    return (B) b;
   }
 
   @Test
   public void testTimeout() throws Exception {
     SolrQuery q = new SolrQuery("*:*");
     try (Http2SolrClient client =
-        getHttp2SolrClientBuilder(
-                getBaseUrl() + SLOW_SERVLET_PATH, DEFAULT_CONNECTION_TIMEOUT, 2000)
-            .withDefaultCollection(DEFAULT_CORE)
-            .build()) {
+        (Http2SolrClient)
+            builder(getBaseUrl() + SLOW_SERVLET_PATH, DEFAULT_CONNECTION_TIMEOUT, 2000)
+                .withDefaultCollection(DEFAULT_CORE)
+                .build()) {
       client.query(q, SolrRequest.METHOD.GET);
       fail("No exception thrown.");
     } catch (SolrServerException e) {
@@ -230,9 +73,10 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   public void test0IdleTimeout() throws Exception {
     SolrQuery q = new SolrQuery("*:*");
     try (Http2SolrClient client =
-        getHttp2SolrClientBuilder(getBaseUrl() + DEBUG_SERVLET_PATH, DEFAULT_CONNECTION_TIMEOUT, 0)
-            .withDefaultCollection(DEFAULT_CORE)
-            .build()) {
+        (Http2SolrClient)
+            builder(getBaseUrl() + DEBUG_SERVLET_PATH, DEFAULT_CONNECTION_TIMEOUT, 0)
+                .withDefaultCollection(DEFAULT_CORE)
+                .build()) {
       try {
         client.query(q, SolrRequest.METHOD.GET);
       } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
@@ -244,10 +88,11 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   public void testRequestTimeout() throws Exception {
     SolrQuery q = new SolrQuery("*:*");
     try (Http2SolrClient client =
-        getHttp2SolrClientBuilder(getBaseUrl() + SLOW_SERVLET_PATH, DEFAULT_CONNECTION_TIMEOUT, 0)
-            .withDefaultCollection(DEFAULT_CORE)
-            .withRequestTimeout(500, TimeUnit.MILLISECONDS)
-            .build()) {
+        (Http2SolrClient)
+            builder(getBaseUrl() + SLOW_SERVLET_PATH, DEFAULT_CONNECTION_TIMEOUT, 0)
+                .withDefaultCollection(DEFAULT_CORE)
+                .withRequestTimeout(500, TimeUnit.MILLISECONDS)
+                .build()) {
       client.query(q, SolrRequest.METHOD.GET);
       fail("No exception thrown.");
     } catch (SolrServerException e) {
@@ -261,190 +106,78 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
    */
   @Test
   public void testSolrExceptionCodeNotFromSolr() throws IOException, SolrServerException {
-    final int status = 527;
-    assertEquals(
-        status
-            + " didn't generate an UNKNOWN error code, someone modified the list of valid ErrorCode's w/o changing this test to work a different way",
-        SolrException.ErrorCode.UNKNOWN,
-        SolrException.ErrorCode.getErrorCode(status));
-
     try (Http2SolrClient client =
         new Http2SolrClient.Builder(getBaseUrl() + DEBUG_SERVLET_PATH)
             .withDefaultCollection(DEFAULT_CORE)
             .build()) {
-      DebugServlet.setErrorCode(status);
-      try {
-        SolrQuery q = new SolrQuery("foo");
-        client.query(q, SolrRequest.METHOD.GET);
-        fail("Didn't get excepted exception from oversided request");
-      } catch (SolrException e) {
-        assertEquals("Unexpected exception status code", status, e.code());
-      }
+      super.testSolrExceptionCodeNotFromSolr(client);
     } finally {
       DebugServlet.clear();
     }
   }
 
-  /**
-   * test that SolrExceptions thrown by HttpSolrClient can correctly encapsulate http status codes
-   * even when not on the list of ErrorCodes solr may return.
-   */
   @Test
   public void testSolrExceptionWithNullBaseurl() throws IOException, SolrServerException {
-    final int status = 527;
-    assertEquals(
-        status
-            + " didn't generate an UNKNOWN error code, someone modified the list of valid ErrorCode's w/o changing this test to work a different way",
-        SolrException.ErrorCode.UNKNOWN,
-        SolrException.ErrorCode.getErrorCode(status));
-
     try (Http2SolrClient client = new Http2SolrClient.Builder(null).build()) {
-      DebugServlet.setErrorCode(status);
-      try {
-        // if client base url is null, request url will be used in exception message
-        SolrPing ping = new SolrPing();
-        ping.setBasePath(getBaseUrl() + DEBUG_SERVLET_PATH);
-        client.request(ping, DEFAULT_CORE);
-
-        fail("Didn't get excepted exception from oversided request");
-      } catch (SolrException e) {
-        assertEquals("Unexpected exception status code", status, e.code());
-        assertTrue(e.getMessage().contains(getBaseUrl()));
-      }
+      super.testSolrExceptionWithNullBaseurl(client);
     } finally {
       DebugServlet.clear();
     }
   }
 
-  @Test
-  public void testQuery() throws Exception {
+  @Override
+  protected void testQuerySetup(SolrRequest.METHOD method, ResponseParser rp) throws Exception {
     DebugServlet.clear();
     String url = getBaseUrl() + DEBUG_SERVLET_PATH;
     SolrQuery q = new SolrQuery("foo");
     q.setParam("a", "\u1234");
-    try (Http2SolrClient client =
-        new Http2SolrClient.Builder(url).withDefaultCollection(DEFAULT_CORE).build()) {
-
-      try {
-        client.query(q, SolrRequest.METHOD.GET);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-
-      // default method
-      assertEquals("get", DebugServlet.lastMethod);
-      // agent
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      // default wt
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
-      assertEquals("javabin", DebugServlet.parameters.get(CommonParams.WT)[0]);
-      // default version
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
-      assertEquals(
-          client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
-      // agent
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      // content-type
-      assertNull(DebugServlet.headers.get("content-type"));
-      // param encoding
-      assertEquals(1, DebugServlet.parameters.get("a").length);
-      assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
-
-      // POST
-      DebugServlet.clear();
-      try {
-        client.query(q, SolrRequest.METHOD.POST);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-
-      assertEquals("post", DebugServlet.lastMethod);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
-      assertEquals("javabin", DebugServlet.parameters.get(CommonParams.WT)[0]);
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
-      assertEquals(
-          client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
-      assertEquals(1, DebugServlet.parameters.get("a").length);
-      assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals("application/x-www-form-urlencoded", DebugServlet.headers.get("content-type"));
-
-      // PUT
-      DebugServlet.clear();
-      try {
-        client.query(q, SolrRequest.METHOD.PUT);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-
-      assertEquals("put", DebugServlet.lastMethod);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
-      assertEquals("javabin", DebugServlet.parameters.get(CommonParams.WT)[0]);
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
+    Http2SolrClient.Builder b =
+        new Http2SolrClient.Builder(url).withDefaultCollection(DEFAULT_CORE);
+    if (rp != null) {
+      b.withResponseParser(rp);
+    }
+    try (Http2SolrClient client = b.build()) {
+      client.query(q, method);
       assertEquals(
           client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
-      assertEquals(1, DebugServlet.parameters.get("a").length);
-      assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals("application/x-www-form-urlencoded", DebugServlet.headers.get("content-type"));
+    } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
     }
-    // XML/GET
-    try (Http2SolrClient client =
-        new Http2SolrClient.Builder(url).withResponseParser(new XMLResponseParser()).build()) {
+  }
 
-      DebugServlet.clear();
-      try {
-        client.query(q, SolrRequest.METHOD.GET);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
+  @Test
+  @Override
+  public void testQueryGet() throws Exception {
+    super.testQueryGet();
+  }
 
-      assertEquals("get", DebugServlet.lastMethod);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
-      assertEquals("xml", DebugServlet.parameters.get(CommonParams.WT)[0]);
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
-      assertEquals(
-          client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
-      assertEquals(1, DebugServlet.parameters.get("a").length);
-      assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
+  @Test
+  @Override
+  public void testQueryPost() throws Exception {
+    super.testQueryPost();
+  }
 
-      // XML/POST
-      DebugServlet.clear();
-      try {
-        client.query(q, SolrRequest.METHOD.POST);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
+  @Test
+  @Override
+  public void testQueryPut() throws Exception {
+    super.testQueryPut();
+  }
 
-      assertEquals("post", DebugServlet.lastMethod);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
-      assertEquals("xml", DebugServlet.parameters.get(CommonParams.WT)[0]);
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
-      assertEquals(
-          client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
-      assertEquals(1, DebugServlet.parameters.get("a").length);
-      assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals("application/x-www-form-urlencoded", DebugServlet.headers.get("content-type"));
+  @Test
+  @Override
+  public void testQueryXmlGet() throws Exception {
+    super.testQueryXmlGet();
+  }
 
-      DebugServlet.clear();
-      try {
-        client.query(q, SolrRequest.METHOD.PUT);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
+  @Test
+  @Override
+  public void testQueryXmlPost() throws Exception {
+    super.testQueryXmlPost();
+  }
 
-      assertEquals("put", DebugServlet.lastMethod);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
-      assertEquals("xml", DebugServlet.parameters.get(CommonParams.WT)[0]);
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
-      assertEquals(
-          client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
-      assertEquals(1, DebugServlet.parameters.get("a").length);
-      assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals("application/x-www-form-urlencoded", DebugServlet.headers.get("content-type"));
-    }
+  @Test
+  @Override
+  public void testQueryXmlPut() throws Exception {
+    super.testQueryXmlPut();
   }
 
   @Test
@@ -457,41 +190,30 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
         client.deleteById("id");
       } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
       }
-
-      // default method
-      assertEquals("post", DebugServlet.lastMethod);
-      // agent
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      // default wt
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
-      assertEquals("javabin", DebugServlet.parameters.get(CommonParams.WT)[0]);
-      // default version
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
       assertEquals(
           client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
-      // agent
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
+      assertEquals("javabin", DebugServlet.parameters.get(CommonParams.WT)[0]);
+      validateDelete();
     }
-    // XML
+  }
+
+  @Test
+  public void testDeleteXml() throws Exception {
+    DebugServlet.clear();
+    String url = getBaseUrl() + "/debug/foo";
     try (Http2SolrClient client =
         new Http2SolrClient.Builder(url)
             .withDefaultCollection(DEFAULT_CORE)
             .withResponseParser(new XMLResponseParser())
             .build()) {
-
       try {
         client.deleteByQuery("*:*");
       } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
       }
-
-      assertEquals("post", DebugServlet.lastMethod);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
-      assertEquals("xml", DebugServlet.parameters.get(CommonParams.WT)[0]);
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
       assertEquals(
           client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
+      assertEquals("xml", DebugServlet.parameters.get(CommonParams.WT)[0]);
+      validateDelete();
     }
   }
 
@@ -502,110 +224,42 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
         new Http2SolrClient.Builder(getBaseUrl() + DEBUG_SERVLET_PATH)
             .withDefaultCollection(DEFAULT_CORE)
             .build()) {
-      Collection<String> ids = Collections.singletonList("a");
-      try {
-        client.getById("a");
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-
-      try {
-        client.getById(ids, null);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-
-      try {
-        client.getById("foo", "a");
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-
-      try {
-        client.getById("foo", ids, null);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
+      super.testGetById(client);
     }
   }
 
   @Test
-  public void testUpdate() throws Exception {
-    DebugServlet.clear();
+  public void testUpdateDefault() throws Exception {
     String url = getBaseUrl() + DEBUG_SERVLET_PATH;
-    UpdateRequest req = new UpdateRequest();
-    req.add(new SolrInputDocument());
-    req.setParam("a", "\u1234");
     try (Http2SolrClient client =
         new Http2SolrClient.Builder(url).withDefaultCollection(DEFAULT_CORE).build()) {
-
-      try {
-        client.request(req);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-
-      // default method
-      assertEquals("post", DebugServlet.lastMethod);
-      // agent
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      // default wt
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
-      assertEquals("javabin", DebugServlet.parameters.get(CommonParams.WT)[0]);
-      // default version
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
-      assertEquals(
-          client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
-      // content type
-      assertEquals("application/javabin", DebugServlet.headers.get("content-type"));
-      // parameter encoding
-      assertEquals(1, DebugServlet.parameters.get("a").length);
-      assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
+      testUpdate(client, WT.JAVABIN, "application/javabin", "\u1234");
     }
+  }
+
+  @Test
+  public void testUpdateXml() throws Exception {
+    String url = getBaseUrl() + "/debug/foo";
     try (Http2SolrClient client =
         new Http2SolrClient.Builder(url)
             .withDefaultCollection(DEFAULT_CORE)
             .withRequestWriter(new RequestWriter())
             .withResponseParser(new XMLResponseParser())
             .build()) {
-
-      // XML response and writer
-      try {
-        client.request(req);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-
-      assertEquals("post", DebugServlet.lastMethod);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
-      assertEquals("xml", DebugServlet.parameters.get(CommonParams.WT)[0]);
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
-      assertEquals(
-          client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
-      assertEquals("application/xml; charset=UTF-8", DebugServlet.headers.get("content-type"));
-      assertEquals(1, DebugServlet.parameters.get("a").length);
-      assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
+      testUpdate(client, WT.XML, "application/xml; charset=UTF-8", "\u1234");
     }
+  }
 
-    // javabin request
+  @Test
+  public void testUpdateJavabin() throws Exception {
+    String url = getBaseUrl() + "/debug/foo";
     try (Http2SolrClient client =
         new Http2SolrClient.Builder(url)
             .withDefaultCollection(DEFAULT_CORE)
             .withRequestWriter(new BinaryRequestWriter())
             .withResponseParser(new BinaryResponseParser())
             .build()) {
-
-      DebugServlet.clear();
-      try {
-        client.request(req);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-
-      assertEquals("post", DebugServlet.lastMethod);
-      assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
-      assertEquals("javabin", DebugServlet.parameters.get(CommonParams.WT)[0]);
-      assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
-      assertEquals(
-          client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
-      assertEquals("application/javabin", DebugServlet.headers.get("content-type"));
-      assertEquals(1, DebugServlet.parameters.get("a").length);
-      assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
+      testUpdate(client, WT.JAVABIN, "application/javabin", "\u1234");
     }
   }
 
@@ -673,126 +327,15 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
 
   @Test
   public void testCollectionParameters() throws IOException, SolrServerException {
-
-    try (Http2SolrClient client = new Http2SolrClient.Builder(getBaseUrl()).build()) {
-      SolrInputDocument doc = new SolrInputDocument();
-      doc.addField("id", "collection");
-      client.add("collection1", doc);
-      client.commit("collection1");
-
-      assertEquals(
-          1,
-          client.query("collection1", new SolrQuery("id:collection")).getResults().getNumFound());
-    }
-
-    final String collection1Url = getCoreUrl();
-    try (Http2SolrClient client = new Http2SolrClient.Builder(collection1Url).build()) {
-      assertEquals(1, client.query(new SolrQuery("id:collection")).getResults().getNumFound());
-    }
-  }
-
-  private void setReqParamsOf(UpdateRequest req, String... keys) {
-    if (keys != null) {
-      for (String k : keys) {
-        req.setParam(k, k + "Value");
-      }
-    }
-  }
-
-  private void verifyServletState(Http2SolrClient client, SolrRequest<?> request) {
-    // check query String
-    Iterator<String> paramNames = request.getParams().getParameterNamesIterator();
-    while (paramNames.hasNext()) {
-      String name = paramNames.next();
-      String[] values = request.getParams().getParams(name);
-      if (values != null) {
-        for (String value : values) {
-          boolean shouldBeInQueryString =
-              client.getUrlParamNames().contains(name)
-                  || (request.getQueryParams() != null && request.getQueryParams().contains(name));
-          assertEquals(
-              shouldBeInQueryString, DebugServlet.queryString.contains(name + "=" + value));
-          // in either case, it should be in the parameters
-          assertNotNull(DebugServlet.parameters.get(name));
-          assertEquals(1, DebugServlet.parameters.get(name).length);
-          assertEquals(value, DebugServlet.parameters.get(name)[0]);
-        }
-      }
-    }
+    Http2SolrClient baseUrlClient = new Http2SolrClient.Builder(getBaseUrl()).build();
+    Http2SolrClient collection1UrlClient = new Http2SolrClient.Builder(getCoreUrl()).build();
+    testCollectionParameters(baseUrlClient, collection1UrlClient);
   }
 
   @Test
+  @Override
   public void testQueryString() throws Exception {
-
-    final String clientUrl = getBaseUrl() + DEBUG_SERVLET_PATH;
-    UpdateRequest req = new UpdateRequest();
-
-    try (Http2SolrClient client =
-        new Http2SolrClient.Builder(clientUrl)
-            .withDefaultCollection(DEFAULT_CORE)
-            .withTheseParamNamesInTheUrl(Set.of("serverOnly"))
-            .build()) {
-      // test without request query params
-      DebugServlet.clear();
-      setReqParamsOf(req, "serverOnly", "notServer");
-
-      try {
-        client.request(req);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-      verifyServletState(client, req);
-
-      // test without server query params
-      DebugServlet.clear();
-    }
-    try (Http2SolrClient client =
-        new Http2SolrClient.Builder(clientUrl).withTheseParamNamesInTheUrl(Set.of()).build()) {
-      req = new UpdateRequest();
-      req.setQueryParams(Set.of("requestOnly"));
-      setReqParamsOf(req, "requestOnly", "notRequest");
-      try {
-        client.request(req);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-      verifyServletState(client, req);
-
-      // test with both request and server query params
-      DebugServlet.clear();
-    }
-    try (Http2SolrClient client =
-        new Http2SolrClient.Builder(clientUrl)
-            .withTheseParamNamesInTheUrl(Set.of("serverOnly", "both"))
-            .build()) {
-      req = new UpdateRequest();
-      req.setQueryParams(Set.of("requestOnly", "both"));
-      setReqParamsOf(req, "serverOnly", "requestOnly", "both", "neither");
-      try {
-        client.request(req);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-      verifyServletState(client, req);
-    }
-    try (Http2SolrClient client =
-        new Http2SolrClient.Builder(clientUrl)
-            .withTheseParamNamesInTheUrl(Set.of("serverOnly", "both"))
-            .build()) {
-
-      // test with both request and server query params with single stream
-      DebugServlet.clear();
-      req = new UpdateRequest();
-      req.add(new SolrInputDocument());
-      req.setQueryParams(Set.of("requestOnly", "both"));
-      setReqParamsOf(req, "serverOnly", "requestOnly", "both", "neither");
-      try {
-        client.request(req);
-      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
-      }
-      // NOTE: single stream requests send all the params
-      // as part of the query string.  So add "neither" to the request
-      // so it passes the verification step.
-      req.setQueryParams(Set.of("requestOnly", "both", "neither"));
-      verifyServletState(client, req);
-    }
+    super.testQueryString();
   }
 
   @Test
@@ -843,18 +386,6 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
         e.getMessage().contains(expectedMessage));
   }
 
-  @Test
-  public void testBadExplicitCredentials() {
-    expectThrowsAndMessage(
-        IllegalStateException.class,
-        () -> new Http2SolrClient.Builder().withBasicAuthCredentials("foo", null),
-        "Invalid Authentication credentials");
-    expectThrowsAndMessage(
-        IllegalStateException.class,
-        () -> new Http2SolrClient.Builder().withBasicAuthCredentials(null, "foo"),
-        "Invalid Authentication credentials");
-  }
-
   @Test
   public void testSetCredentialsExplicitly() {
     try (Http2SolrClient client =
@@ -862,23 +393,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
             .withDefaultCollection(DEFAULT_CORE)
             .withBasicAuthCredentials("foo", "explicit")
             .build(); ) {
-      QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
-      try {
-        ignoreException("Error from server");
-        client.request(r);
-      } catch (Exception e) {
-        // expected
-      }
-      unIgnoreException("Error from server");
-      assertTrue(DebugServlet.headers.size() > 0);
-      String authorizationHeader = DebugServlet.headers.get("authorization");
-      assertNotNull(
-          "No authorization information in headers found. Headers: " + DebugServlet.headers,
-          authorizationHeader);
-      assertEquals(
-          "Basic "
-              + Base64.getEncoder().encodeToString("foo:explicit".getBytes(StandardCharsets.UTF_8)),
-          authorizationHeader);
+      super.testSetCredentialsExplicitly(client);
     }
   }
 
@@ -922,33 +437,13 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   }
 
   @Test
-  public void testPerRequestCredentialsWin() {
+  public void testPerRequestCredentials() {
     try (Http2SolrClient client =
         new Http2SolrClient.Builder(getBaseUrl() + DEBUG_SERVLET_PATH)
             .withDefaultCollection(DEFAULT_CORE)
             .withBasicAuthCredentials("foo2", "explicit")
             .build(); ) {
-      QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
-      r.setBasicAuthCredentials("foo3", "per-request");
-      try {
-        ignoreException("Error from server");
-        client.request(r);
-      } catch (Exception e) {
-        // expected
-      }
-      unIgnoreException("Error from server");
-      assertTrue(DebugServlet.headers.size() > 0);
-      String authorizationHeader = DebugServlet.headers.get("authorization");
-      assertNotNull(
-          "No authorization information in headers found. Headers: " + DebugServlet.headers,
-          authorizationHeader);
-      assertEquals(
-          "Basic "
-              + Base64.getEncoder()
-                  .encodeToString("foo3:per-request".getBytes(StandardCharsets.UTF_8)),
-          authorizationHeader);
-    } finally {
-      System.clearProperty("basicauth");
+      super.testPerRequestCredentials(client);
     }
   }
 
@@ -957,18 +452,8 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
     try (Http2SolrClient client =
         new Http2SolrClient.Builder(getBaseUrl() + DEBUG_SERVLET_PATH)
             .withDefaultCollection(DEFAULT_CORE)
-            .build()) {
-      QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
-      try {
-        ignoreException("Error from server");
-        client.request(r);
-      } catch (Exception e) {
-        // expected
-      }
-      unIgnoreException("Error from server");
-      assertFalse(
-          "Expecting no authorization header but got: " + DebugServlet.headers,
-          DebugServlet.headers.containsKey("authorization"));
+            .build(); ) {
+      super.testNoCredentials(client);
     }
   }
 
@@ -980,48 +465,18 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
             .withDefaultCollection(DEFAULT_CORE)
             .withOptionalBasicAuthCredentials("foo:expli:cit")
             .build(); ) {
-      QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
-      try {
-        ignoreException("Error from server");
-        client.request(r);
-      } catch (Exception e) {
-        // expected
-      }
-      unIgnoreException("Error from server");
-      assertTrue(DebugServlet.headers.size() > 0);
-      String authorizationHeader = DebugServlet.headers.get("authorization");
-      assertNotNull(
-          "No authorization information in headers found. Headers: " + DebugServlet.headers,
-          authorizationHeader);
-      assertEquals(
-          "Basic "
-              + Base64.getEncoder()
-                  .encodeToString("foo:expli:cit".getBytes(StandardCharsets.UTF_8)),
-          authorizationHeader);
+      super.testUseOptionalCredentials(client);
     }
   }
 
   @Test
   public void testUseOptionalCredentialsWithNull() {
-    // username foo, password with embedded colon separator is "expli:cit".
     try (Http2SolrClient client =
         new Http2SolrClient.Builder(getBaseUrl() + DEBUG_SERVLET_PATH)
             .withDefaultCollection(DEFAULT_CORE)
             .withOptionalBasicAuthCredentials(null)
             .build(); ) {
-      QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
-      try {
-        ignoreException("Error from server");
-        client.request(r);
-      } catch (Exception e) {
-        // expected
-      }
-      unIgnoreException("Error from server");
-      assertTrue(DebugServlet.headers.size() > 0);
-      String authorizationHeader = DebugServlet.headers.get("authorization");
-      assertNull(
-          "No authorization headers expected. Headers: " + DebugServlet.headers,
-          authorizationHeader);
+      super.testUseOptionalCredentialsWithNull(client);
     }
   }
 
@@ -1039,6 +494,18 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
         "Invalid Authentication credential formatting. Provide username and password in the 'username:password' format.");
   }
 
+  @Test
+  public void testBadExplicitCredentials() {
+    expectThrowsAndMessage(
+        IllegalStateException.class,
+        () -> new Http2SolrClient.Builder().withBasicAuthCredentials("foo", null),
+        "Invalid Authentication credentials");
+    expectThrowsAndMessage(
+        IllegalStateException.class,
+        () -> new Http2SolrClient.Builder().withBasicAuthCredentials(null, "foo"),
+        "Invalid Authentication credentials");
+  }
+
   @Test
   public void testBadHttpFactory() {
     System.setProperty(HttpClientUtil.SYS_PROP_HTTP_CLIENT_BUILDER_FACTORY, "FakeClassName");
@@ -1055,21 +522,15 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
 
   @Test
   public void testGetRawStream() throws Exception {
-    DebugServlet.clear();
     try (Http2SolrClient client =
-        getHttp2SolrClientBuilder(
-                getBaseUrl() + DEBUG_SERVLET_PATH,
-                DEFAULT_CONNECTION_TIMEOUT,
-                DEFAULT_CONNECTION_TIMEOUT)
-            .withDefaultCollection(DEFAULT_CORE)
-            .build()) {
-      final var req = new QueryRequest(params("q", "*:*"));
-      req.setResponseParser(new InputStreamResponseParser("xml"));
-      final var rsp = req.process(client);
-      Object stream = rsp.getResponse().get("stream");
-      assertNotNull(stream);
-      MatcherAssert.assertThat(stream, instanceOf(InputStream.class));
-      org.apache.solr.common.util.IOUtils.closeQuietly((InputStream) stream);
+        (Http2SolrClient)
+            builder(
+                    getBaseUrl() + DEBUG_SERVLET_PATH,
+                    DEFAULT_CONNECTION_TIMEOUT,
+                    DEFAULT_CONNECTION_TIMEOUT)
+                .withDefaultCollection(DEFAULT_CORE)
+                .build()) {
+      super.testGetRawStream(client);
     }
   }
 
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpJdkSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpJdkSolrClientTest.java
new file mode 100644
index 00000000000..5c614921762
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpJdkSolrClientTest.java
@@ -0,0 +1,599 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.impl;
+
+import java.io.IOException;
+import java.net.CookieHandler;
+import java.net.CookieManager;
+import java.net.Socket;
+import java.net.http.HttpClient;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.Collections;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.X509ExtendedTrustManager;
+import org.apache.lucene.util.NamedThreadFactory;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.ResponseParser;
+import org.apache.solr.client.solrj.SolrQuery;
+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.response.SolrPingResponse;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.util.SSLTestConfig;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class HttpJdkSolrClientTest extends HttpSolrClientTestBase {
+
+  private static SSLContext allTrustingSslContext;
+
+  @BeforeClass
+  public static void beforeClass() {
+    try {
+      KeyManagerFactory keyManagerFactory =
+          KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+      SSLTestConfig stc = SolrTestCaseJ4.sslConfig;
+      keyManagerFactory.init(stc.defaultKeyStore(), stc.defaultKeyStorePassword().toCharArray());
+
+      SSLContext sslContext = SSLContext.getInstance("SSL");
+      sslContext.init(
+          keyManagerFactory.getKeyManagers(),
+          new TrustManager[] {MOCK_TRUST_MANAGER},
+          stc.notSecureSecureRandom());
+      allTrustingSslContext = sslContext;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @After
+  public void workaroundToReleaseThreads_noClosableUntilJava21() {
+    System.gc();
+  }
+
+  @Test
+  @Override
+  public void testQueryGet() throws Exception {
+    super.testQueryGet();
+  }
+
+  @Test
+  @Override
+  public void testQueryPost() throws Exception {
+    super.testQueryPost();
+  }
+
+  @Test
+  @Override
+  public void testQueryPut() throws Exception {
+    super.testQueryPut();
+  }
+
+  @Test
+  @Override
+  public void testQueryXmlGet() throws Exception {
+    super.testQueryXmlGet();
+  }
+
+  @Test
+  @Override
+  public void testQueryXmlPost() throws Exception {
+    super.testQueryXmlPost();
+  }
+
+  @Test
+  @Override
+  public void testQueryXmlPut() throws Exception {
+    super.testQueryXmlPut();
+  }
+
+  @Test
+  public void testDelete() throws Exception {
+    DebugServlet.clear();
+    String url = getBaseUrl() + DEBUG_SERVLET_PATH;
+    try (HttpJdkSolrClient client = builder(url).build()) {
+      try {
+        client.deleteById("id");
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+      }
+      assertEquals(
+          client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
+      assertEquals("javabin", DebugServlet.parameters.get(CommonParams.WT)[0]);
+      validateDelete();
+    }
+  }
+
+  @Test
+  public void testDeleteXml() throws Exception {
+    DebugServlet.clear();
+    String url = getBaseUrl() + DEBUG_SERVLET_PATH;
+    try (HttpJdkSolrClient client =
+        builder(url).withResponseParser(new XMLResponseParser()).build()) {
+      try {
+        client.deleteByQuery("*:*");
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+      }
+      assertEquals(
+          client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
+      assertEquals("xml", DebugServlet.parameters.get(CommonParams.WT)[0]);
+      validateDelete();
+    }
+  }
+
+  @Override
+  protected void testQuerySetup(SolrRequest.METHOD method, ResponseParser rp) throws Exception {
+    DebugServlet.clear();
+    if (rp instanceof XMLResponseParser) {
+      DebugServlet.addResponseHeader("Content-Type", "application/xml; charset=UTF-8");
+      DebugServlet.responseBody = "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n<response />";
+    } else {
+      DebugServlet.addResponseHeader("Content-Type", "application/octet-stream");
+      DebugServlet.responseBody = javabinResponse();
+    }
+    String url = getBaseUrl() + DEBUG_SERVLET_PATH;
+    SolrQuery q = new SolrQuery("foo");
+    q.setParam("a", "\u1234");
+    HttpJdkSolrClient.Builder b = builder(url);
+    if (rp != null) {
+      b.withResponseParser(rp);
+    }
+    try (HttpJdkSolrClient client = b.build()) {
+      client.query(q, method);
+      assertEquals(
+          client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
+    }
+  }
+
+  @Test
+  public void testGetById() throws Exception {
+    DebugServlet.clear();
+    try (HttpJdkSolrClient client = builder(getBaseUrl() + DEBUG_SERVLET_PATH).build()) {
+      super.testGetById(client);
+    }
+  }
+
+  @Test
+  public void testTimeout() throws Exception {
+    SolrQuery q = new SolrQuery("*:*");
+    try (HttpJdkSolrClient client =
+        (HttpJdkSolrClient) builder(getBaseUrl() + SLOW_SERVLET_PATH, 500, 500).build()) {
+      client.query(q, SolrRequest.METHOD.GET);
+      fail("No exception thrown.");
+    } catch (SolrServerException e) {
+      assertTrue(e.getMessage().contains("timeout") || e.getMessage().contains("Timeout"));
+    }
+  }
+
+  @Test
+  public void test0IdleTimeout() throws Exception {
+    SolrQuery q = new SolrQuery("*:*");
+    try (HttpJdkSolrClient client =
+        (HttpJdkSolrClient)
+            builder(getBaseUrl() + DEBUG_SERVLET_PATH, DEFAULT_CONNECTION_TIMEOUT, 0).build()) {
+      try {
+        client.query(q, SolrRequest.METHOD.GET);
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+      }
+    }
+  }
+
+  @Test
+  public void testRequestTimeout() throws Exception {
+    SolrQuery q = new SolrQuery("*:*");
+    try (HttpJdkSolrClient client =
+        (HttpJdkSolrClient)
+            builder(getBaseUrl() + SLOW_SERVLET_PATH, DEFAULT_CONNECTION_TIMEOUT, 0)
+                .withRequestTimeout(500, TimeUnit.MILLISECONDS)
+                .build()) {
+      client.query(q, SolrRequest.METHOD.GET);
+      fail("No exception thrown.");
+    } catch (SolrServerException e) {
+      assertTrue(e.getMessage().contains("timeout") || e.getMessage().contains("Timeout"));
+    }
+  }
+
+  @Test
+  public void testFollowRedirect() throws Exception {
+    final String clientUrl = getBaseUrl() + REDIRECT_SERVLET_PATH;
+    try (HttpJdkSolrClient client = builder(clientUrl).withFollowRedirects(true).build()) {
+      SolrQuery q = new SolrQuery("*:*");
+      client.query(q);
+    }
+  }
+
+  @Test
+  public void testDoNotFollowRedirect() throws Exception {
+    final String clientUrl = getBaseUrl() + REDIRECT_SERVLET_PATH;
+    try (HttpJdkSolrClient client = builder(clientUrl).withFollowRedirects(false).build()) {
+      SolrQuery q = new SolrQuery("*:*");
+
+      SolrServerException thrown = assertThrows(SolrServerException.class, () -> client.query(q));
+      assertTrue(thrown.getMessage().contains("redirect"));
+    }
+  }
+
+  @Test
+  public void testRedirectSwapping() throws Exception {
+    final String clientUrl = getBaseUrl() + REDIRECT_SERVLET_PATH;
+    SolrQuery q = new SolrQuery("*:*");
+
+    // default for follow redirects is false
+    try (HttpJdkSolrClient client = builder(clientUrl).build()) {
+
+      SolrServerException e = expectThrows(SolrServerException.class, () -> client.query(q));
+      assertTrue(e.getMessage().contains("redirect"));
+    }
+
+    try (HttpJdkSolrClient client = builder(clientUrl).withFollowRedirects(true).build()) {
+      // shouldn't throw an exception
+      client.query(q);
+    }
+
+    // set explicit false for following redirects
+    try (HttpJdkSolrClient client = builder(clientUrl).withFollowRedirects(false).build()) {
+
+      SolrServerException e = expectThrows(SolrServerException.class, () -> client.query(q));
+      assertTrue(e.getMessage().contains("redirect"));
+    }
+  }
+
+  public void testSolrExceptionCodeNotFromSolr() throws IOException, SolrServerException {
+    try (HttpJdkSolrClient client = builder(getBaseUrl() + DEBUG_SERVLET_PATH).build()) {
+      super.testSolrExceptionCodeNotFromSolr(client);
+    } finally {
+      DebugServlet.clear();
+    }
+  }
+
+  @Test
+  public void testSolrExceptionWithNullBaseurl() throws IOException, SolrServerException {
+    try (HttpJdkSolrClient client = builder(null).build()) {
+      super.testSolrExceptionWithNullBaseurl(client);
+    } finally {
+      DebugServlet.clear();
+    }
+  }
+
+  @Test
+  public void testUpdateDefault() throws Exception {
+    String url = getBaseUrl() + DEBUG_SERVLET_PATH;
+    try (HttpJdkSolrClient client = builder(url).build()) {
+      testUpdate(client, WT.JAVABIN, "application/javabin", "\u1234");
+    }
+  }
+
+  @Test
+  public void testUpdateXml() throws Exception {
+    testUpdateXml(false);
+  }
+
+  @Test
+  public void testUpdateXmlWithHttp11() throws Exception {
+    testUpdateXml(true);
+  }
+
+  private void testUpdateXml(boolean http11) throws Exception {
+    String url = getBaseUrl() + DEBUG_SERVLET_PATH;
+
+    // 64k+ post body, just to be sure we are using the [in|out]put streams correctly.
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < 65536; i++) {
+      sb.append("A");
+    }
+    String value = sb.toString();
+
+    try (HttpJdkSolrClient client =
+        builder(url)
+            .withRequestWriter(new RequestWriter())
+            .withResponseParser(new XMLResponseParser())
+            .useHttp1_1(http11)
+            .build()) {
+      testUpdate(client, HttpSolrClientTestBase.WT.XML, "application/xml; charset=UTF-8", value);
+      if (http11) {
+        assertEquals(HttpClient.Version.HTTP_1_1, client.httpClient.version());
+        assertFalse(
+            "The HEAD request should not be performed if already forcing Http/1.1.",
+            client.headRequested);
+      } else {
+        assertEquals(HttpClient.Version.HTTP_2, client.httpClient.version());
+      }
+      assertNoHeadRequestWithSsl(client);
+    }
+  }
+
+  @Test
+  public void testUpdateJavabin() throws Exception {
+    String url = getBaseUrl() + DEBUG_SERVLET_PATH;
+    try (HttpJdkSolrClient client =
+        builder(url)
+            .withRequestWriter(new BinaryRequestWriter())
+            .withResponseParser(new BinaryResponseParser())
+            .build()) {
+      testUpdate(client, WT.JAVABIN, "application/javabin", "\u1234");
+      assertNoHeadRequestWithSsl(client);
+    }
+  }
+
+  @Test
+  public void testCollectionParameters() throws IOException, SolrServerException {
+    HttpJdkSolrClient baseUrlClient = builder(getBaseUrl()).withDefaultCollection(null).build();
+    HttpJdkSolrClient collection1UrlClient =
+        builder(getCoreUrl()).withDefaultCollection(null).build();
+    testCollectionParameters(baseUrlClient, collection1UrlClient);
+  }
+
+  @Test
+  @Override
+  public void testQueryString() throws Exception {
+    super.testQueryString();
+  }
+
+  @Test
+  public void testGetRawStream() throws Exception {
+    try (HttpJdkSolrClient client =
+        (HttpJdkSolrClient)
+            builder(
+                    getBaseUrl() + DEBUG_SERVLET_PATH,
+                    DEFAULT_CONNECTION_TIMEOUT,
+                    DEFAULT_CONNECTION_TIMEOUT)
+                .build()) {
+      super.testGetRawStream(client);
+    }
+  }
+
+  @Test
+  public void testSetCredentialsExplicitly() throws Exception {
+    try (HttpJdkSolrClient client =
+        builder(getBaseUrl() + DEBUG_SERVLET_PATH)
+            .withBasicAuthCredentials("foo", "explicit")
+            .build(); ) {
+      super.testSetCredentialsExplicitly(client);
+    }
+  }
+
+  @Test
+  public void testPerRequestCredentials() throws Exception {
+    try (HttpJdkSolrClient client =
+        builder(getBaseUrl() + DEBUG_SERVLET_PATH)
+            .withBasicAuthCredentials("foo2", "explicit")
+            .build(); ) {
+      super.testPerRequestCredentials(client);
+    }
+  }
+
+  @Test
+  public void testNoCredentials() throws Exception {
+    try (HttpJdkSolrClient client = builder(getBaseUrl() + DEBUG_SERVLET_PATH).build(); ) {
+      super.testNoCredentials(client);
+    }
+  }
+
+  @Test
+  public void testUseOptionalCredentials() throws Exception {
+    // username foo, password with embedded colon separator is "expli:cit".
+    try (HttpJdkSolrClient client =
+        builder(getBaseUrl() + DEBUG_SERVLET_PATH)
+            .withOptionalBasicAuthCredentials("foo:expli:cit")
+            .build(); ) {
+      super.testUseOptionalCredentials(client);
+    }
+  }
+
+  @Test
+  public void testUseOptionalCredentialsWithNull() throws Exception {
+    try (HttpJdkSolrClient client =
+        builder(getBaseUrl() + DEBUG_SERVLET_PATH)
+            .withOptionalBasicAuthCredentials(null)
+            .build(); ) {
+      super.testUseOptionalCredentialsWithNull(client);
+    }
+  }
+
+  @Test
+  public void testProcessorMimeTypes() throws Exception {
+    ResponseParser rp = new XMLResponseParser();
+
+    try (HttpJdkSolrClient client = builder(getBaseUrl()).withResponseParser(rp).build()) {
+      assertTrue(client.processorAcceptsMimeType(rp.getContentTypes(), "application/xml"));
+      assertFalse(client.processorAcceptsMimeType(rp.getContentTypes(), "application/json"));
+      queryToHelpJdkReleaseThreads(client);
+    }
+
+    rp = new BinaryResponseParser();
+    try (HttpJdkSolrClient client = builder(getBaseUrl()).withResponseParser(rp).build()) {
+      assertTrue(
+          client.processorAcceptsMimeType(
+              rp.getContentTypes(), "application/vnd.apache.solr.javabin"));
+      assertTrue(client.processorAcceptsMimeType(rp.getContentTypes(), "application/octet-stream"));
+      assertFalse(client.processorAcceptsMimeType(rp.getContentTypes(), "application/xml"));
+      queryToHelpJdkReleaseThreads(client);
+    }
+  }
+
+  @Test
+  public void testContentTypeToEncoding() throws Exception {
+    try (HttpJdkSolrClient client = builder(getBaseUrl()).build()) {
+      assertEquals("UTF-8", client.contentTypeToEncoding("application/xml; charset=UTF-8"));
+      assertNull(client.contentTypeToEncoding("application/vnd.apache.solr.javabin"));
+      assertNull(client.contentTypeToEncoding("application/octet-stream"));
+      assertNull(client.contentTypeToEncoding("multipart/form-data; boundary=something"));
+      queryToHelpJdkReleaseThreads(client);
+    }
+  }
+
+  @Test
+  public void testPassedInExecutorNotShutdown() throws Exception {
+    ExecutorService myExecutor = null;
+    try {
+      myExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new NamedThreadFactory("tpiens"));
+      try (HttpJdkSolrClient client = builder(getBaseUrl()).withExecutor(myExecutor).build()) {
+        assertEquals(myExecutor, client.executor);
+        queryToHelpJdkReleaseThreads(client);
+      }
+      assertFalse(myExecutor.isShutdown());
+    } finally {
+      try {
+        myExecutor.shutdownNow();
+      } catch (Exception e1) {
+        // ignore
+      }
+    }
+  }
+
+  @Test
+  public void testCookieHandlerSettingHonored() throws Exception {
+    CookieHandler myCookieHandler = new CookieManager();
+    try (HttpJdkSolrClient client =
+        builder(getBaseUrl()).withCookieHandler(myCookieHandler).build()) {
+      assertEquals(myCookieHandler, client.httpClient.cookieHandler().get());
+      queryToHelpJdkReleaseThreads(client);
+    }
+  }
+
+  @Test
+  public void testPing() throws Exception {
+    try (HttpJdkSolrClient client = builder(getBaseUrl()).build()) {
+      SolrPingResponse spr = client.ping("collection1");
+      assertEquals(0, spr.getStatus());
+      assertNull(spr.getException());
+    }
+  }
+
+  /**
+   * This is not required for any test, but there appears to be a bug in the JDK client where it
+   * does not release all threads if the client has not performed any queries, even after a forced
+   * full gc (see "after" in this test class).
+   *
+   * @param client the client
+   */
+  private void queryToHelpJdkReleaseThreads(HttpJdkSolrClient client) throws Exception {
+    client.query("collection1", new MapSolrParams(Collections.singletonMap("q", "*:*")));
+  }
+
+  private void assertNoHeadRequestWithSsl(HttpJdkSolrClient client) {
+    if (isSSLMode()) {
+      assertFalse("The HEAD request should not be performed if using SSL.", client.headRequested);
+    }
+  }
+
+  @Override
+  protected String expectedUserAgent() {
+    return "Solr[" + HttpJdkSolrClient.class.getName() + "] 1.0";
+  }
+
+  @Override
+  @SuppressWarnings(value = "unchecked")
+  protected <B extends HttpSolrClientBuilderBase<?, ?>> B builder(
+      String url, int connectionTimeout, int socketTimeout) {
+    HttpJdkSolrClient.Builder b =
+        new HttpJdkSolrClient.Builder(url)
+            .withConnectionTimeout(connectionTimeout, TimeUnit.MILLISECONDS)
+            .withIdleTimeout(socketTimeout, TimeUnit.MILLISECONDS)
+            .withDefaultCollection(DEFAULT_CORE)
+            .withSSLContext(allTrustingSslContext);
+    return (B) b;
+  }
+
+  private HttpJdkSolrClient.Builder builder(String url) {
+    return builder(url, DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT);
+  }
+
+  private byte[] javabinResponse() {
+    String[] str = JAVABIN_STR.split(" ");
+    byte[] bytes = new byte[str.length];
+    for (int i = 0; i < str.length; i++) {
+      int asInt = 0;
+      bytes[i] = (byte) Integer.decode("#" + str[i]).intValue();
+    }
+    return bytes;
+  }
+
+  private static final String JAVABIN_STR =
+      "02 A2 e0 2e 72 65 73 70 6f "
+          + "6e 73 65 48 65 61 64 65 72 "
+          + "a4 e0 2b 7a 6b 43 6f 6e 6e "
+          + "65 63 74 65 64 01 e0 26 73 "
+          + "74 61 74 75 73 06 00 00 00 "
+          + "00 e0 25 51 54 69 6d 65 06 "
+          + "00 00 00 00 e0 26 70 61 72 "
+          + "61 6d 73 a4 e0 21 71 21 7a "
+          + "e0 24 72 6f 77 73 21 30 e0 "
+          + "22 77 74 27 6a 61 76 61 62 "
+          + "69 6e e0 27 76 65 72 73 69 "
+          + "6f 6e 21 32 e0 28 72 65 73 "
+          + "70 6f 6e 73 65 0c 84 60 60 "
+          + "00 01 80";
+
+  /**
+   * Taken from: https://www.baeldung.com/java-httpclient-ssl sec 4.1, 2024/02/12. This is an
+   * all-trusting Trust Manager. Works with self-signed certificates.
+   */
+  private static final TrustManager MOCK_TRUST_MANAGER =
+      new X509ExtendedTrustManager() {
+        @Override
+        public void checkClientTrusted(X509Certificate[] chain, String authType, Socket socket)
+            throws CertificateException {
+          // no-op
+        }
+
+        @Override
+        public void checkServerTrusted(X509Certificate[] chain, String authType, Socket socket)
+            throws CertificateException {
+          // no-op
+        }
+
+        @Override
+        public void checkClientTrusted(X509Certificate[] chain, String authType, SSLEngine engine)
+            throws CertificateException {
+          // no-op
+        }
+
+        @Override
+        public void checkServerTrusted(X509Certificate[] chain, String authType, SSLEngine engine)
+            throws CertificateException {
+          // no-op
+        }
+
+        @Override
+        public java.security.cert.X509Certificate[] getAcceptedIssuers() {
+          return new java.security.cert.X509Certificate[0];
+        }
+
+        @Override
+        public void checkClientTrusted(X509Certificate[] chain, String authType)
+            throws CertificateException {
+          // no-op
+        }
+
+        @Override
+        public void checkServerTrusted(java.security.cert.X509Certificate[] chain, String authType)
+            throws CertificateException {
+          // no-op
+        }
+      };
+}
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientTestBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientTestBase.java
new file mode 100644
index 00000000000..a6a0ab2a7d6
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientTestBase.java
@@ -0,0 +1,535 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.impl;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Locale;
+import java.util.Set;
+import org.apache.solr.SolrJettyTestBase;
+import org.apache.solr.client.solrj.ResponseParser;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.SolrPing;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.embedded.JettyConfig;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.hamcrest.MatcherAssert;
+import org.junit.BeforeClass;
+
+public abstract class HttpSolrClientTestBase extends SolrJettyTestBase {
+
+  protected static final String DEFAULT_CORE = "foo";
+  protected static final String SLOW_SERVLET_PATH = "/slow";
+  protected static final String SLOW_SERVLET_REGEX = SLOW_SERVLET_PATH + "/*";
+  protected static final String DEBUG_SERVLET_PATH = "/debug";
+  protected static final String DEBUG_SERVLET_REGEX = DEBUG_SERVLET_PATH + "/*";
+  protected static final String REDIRECT_SERVLET_PATH = "/redirect";
+  protected static final String REDIRECT_SERVLET_REGEX = REDIRECT_SERVLET_PATH + "/*";
+
+  @BeforeClass
+  public static void beforeTest() throws Exception {
+    JettyConfig jettyConfig =
+        JettyConfig.builder()
+            .withServlet(
+                new ServletHolder(BasicHttpSolrClientTest.RedirectServlet.class),
+                REDIRECT_SERVLET_REGEX)
+            .withServlet(
+                new ServletHolder(BasicHttpSolrClientTest.SlowServlet.class), SLOW_SERVLET_REGEX)
+            .withServlet(new ServletHolder(DebugServlet.class), DEBUG_SERVLET_REGEX)
+            .withSSLConfig(sslConfig.buildServerSSLConfig())
+            .build();
+    createAndStartJetty(legacyExampleCollection1SolrHome(), jettyConfig);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    System.clearProperty("basicauth");
+    System.clearProperty(HttpClientUtil.SYS_PROP_HTTP_CLIENT_BUILDER_FACTORY);
+    DebugServlet.clear();
+    super.tearDown();
+  }
+
+  protected abstract <B extends HttpSolrClientBuilderBase<?, ?>> B builder(
+      String url, int connectionTimeout, int socketTimeout);
+
+  protected abstract String expectedUserAgent();
+
+  protected abstract void testQuerySetup(SolrRequest.METHOD method, ResponseParser rp)
+      throws Exception;
+
+  public void testQueryGet() throws Exception {
+    testQuerySetup(SolrRequest.METHOD.GET, null);
+    // default method
+    assertEquals("get", DebugServlet.lastMethod);
+    // agent
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    // default wt
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
+    assertEquals("javabin", DebugServlet.parameters.get(CommonParams.WT)[0]);
+    // default version
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
+    // agent
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    // content-type
+    assertNull(DebugServlet.headers.get("content-type"));
+    // param encoding
+    assertEquals(1, DebugServlet.parameters.get("a").length);
+    assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
+  }
+
+  public void testQueryPost() throws Exception {
+    testQuerySetup(SolrRequest.METHOD.POST, null);
+
+    assertEquals("post", DebugServlet.lastMethod);
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
+    assertEquals("javabin", DebugServlet.parameters.get(CommonParams.WT)[0]);
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
+    assertEquals(1, DebugServlet.parameters.get("a").length);
+    assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    assertEquals("application/x-www-form-urlencoded", DebugServlet.headers.get("content-type"));
+  }
+
+  public void testQueryPut() throws Exception {
+    testQuerySetup(SolrRequest.METHOD.PUT, null);
+
+    assertEquals("put", DebugServlet.lastMethod);
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
+    assertEquals("javabin", DebugServlet.parameters.get(CommonParams.WT)[0]);
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
+    assertEquals(1, DebugServlet.parameters.get("a").length);
+    assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    assertEquals("application/x-www-form-urlencoded", DebugServlet.headers.get("content-type"));
+  }
+
+  public void testQueryXmlGet() throws Exception {
+    testQuerySetup(SolrRequest.METHOD.GET, new XMLResponseParser());
+
+    assertEquals("get", DebugServlet.lastMethod);
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
+    assertEquals("xml", DebugServlet.parameters.get(CommonParams.WT)[0]);
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
+    assertEquals(1, DebugServlet.parameters.get("a").length);
+    assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+  }
+
+  public void testQueryXmlPost() throws Exception {
+    testQuerySetup(SolrRequest.METHOD.POST, new XMLResponseParser());
+
+    assertEquals("post", DebugServlet.lastMethod);
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
+    assertEquals("xml", DebugServlet.parameters.get(CommonParams.WT)[0]);
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
+    assertEquals(1, DebugServlet.parameters.get("a").length);
+    assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    assertEquals("application/x-www-form-urlencoded", DebugServlet.headers.get("content-type"));
+  }
+
+  public void testQueryXmlPut() throws Exception {
+    testQuerySetup(SolrRequest.METHOD.PUT, new XMLResponseParser());
+
+    assertEquals("put", DebugServlet.lastMethod);
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
+    assertEquals("xml", DebugServlet.parameters.get(CommonParams.WT)[0]);
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
+    assertEquals(1, DebugServlet.parameters.get("a").length);
+    assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    assertEquals("application/x-www-form-urlencoded", DebugServlet.headers.get("content-type"));
+  }
+
+  protected void validateDelete() {
+    // default method
+    assertEquals("post", DebugServlet.lastMethod);
+    // agent
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    // default wt
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
+    // default version
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
+    // agent
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+  }
+
+  public void testGetById(HttpSolrClientBase client) throws Exception {
+    DebugServlet.clear();
+    Collection<String> ids = Collections.singletonList("a");
+    try {
+      client.getById("a");
+    } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+    }
+
+    try {
+      client.getById(ids, null);
+    } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+    }
+
+    try {
+      client.getById("foo", "a");
+    } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+    }
+
+    try {
+      client.getById("foo", ids, null);
+    } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+    }
+  }
+
+  /**
+   * test that SolrExceptions thrown by HttpSolrClient can correctly encapsulate http status codes
+   * even when not on the list of ErrorCodes solr may return.
+   */
+  public void testSolrExceptionCodeNotFromSolr(HttpSolrClientBase client)
+      throws IOException, SolrServerException {
+    final int status = 527;
+    assertEquals(
+        status
+            + " didn't generate an UNKNOWN error code, someone modified the list of valid ErrorCode's w/o changing this test to work a different way",
+        SolrException.ErrorCode.UNKNOWN,
+        SolrException.ErrorCode.getErrorCode(status));
+
+    DebugServlet.setErrorCode(status);
+    try {
+      SolrQuery q = new SolrQuery("foo");
+      client.query(q, SolrRequest.METHOD.GET);
+      fail("Didn't get excepted exception from oversided request");
+    } catch (SolrException e) {
+      assertEquals("Unexpected exception status code", status, e.code());
+    }
+  }
+
+  /**
+   * test that SolrExceptions thrown by HttpSolrClient can correctly encapsulate http status codes
+   * even when not on the list of ErrorCodes solr may return.
+   */
+  public void testSolrExceptionWithNullBaseurl(HttpSolrClientBase client)
+      throws IOException, SolrServerException {
+    final int status = 527;
+    assertEquals(
+        status
+            + " didn't generate an UNKNOWN error code, someone modified the list of valid ErrorCode's w/o changing this test to work a different way",
+        SolrException.ErrorCode.UNKNOWN,
+        SolrException.ErrorCode.getErrorCode(status));
+
+    DebugServlet.setErrorCode(status);
+    try {
+      // if client base url is null, request url will be used in exception message
+      SolrPing ping = new SolrPing();
+      ping.setBasePath(getBaseUrl() + DEBUG_SERVLET_PATH);
+      client.request(ping, DEFAULT_CORE);
+
+      fail("Didn't get excepted exception from oversided request");
+    } catch (SolrException e) {
+      assertEquals("Unexpected exception status code", status, e.code());
+      assertTrue(e.getMessage().contains(getBaseUrl()));
+    }
+  }
+
+  protected enum WT {
+    JAVABIN,
+    XML
+  };
+
+  protected void testUpdate(HttpSolrClientBase client, WT wt, String contentType, String docIdValue)
+      throws Exception {
+    DebugServlet.clear();
+    UpdateRequest req = new UpdateRequest();
+    SolrInputDocument doc = new SolrInputDocument();
+    doc.addField("id", docIdValue);
+    req.add(doc);
+    req.setParam("a", "\u1234");
+
+    try {
+      client.request(req);
+    } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+    }
+
+    assertEquals("post", DebugServlet.lastMethod);
+    assertEquals(expectedUserAgent(), DebugServlet.headers.get("user-agent"));
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.WT).length);
+    assertEquals(
+        wt.toString().toLowerCase(Locale.ROOT), DebugServlet.parameters.get(CommonParams.WT)[0]);
+    assertEquals(1, DebugServlet.parameters.get(CommonParams.VERSION).length);
+    assertEquals(
+        client.getParser().getVersion(), DebugServlet.parameters.get(CommonParams.VERSION)[0]);
+    assertEquals(contentType, DebugServlet.headers.get("content-type"));
+    assertEquals(1, DebugServlet.parameters.get("a").length);
+    assertEquals("\u1234", DebugServlet.parameters.get("a")[0]);
+
+    if (wt == WT.XML) {
+      String requestBody = new String(DebugServlet.requestBody, StandardCharsets.UTF_8);
+      assertTrue(requestBody, requestBody.contains("<field name=\"id\">" + docIdValue));
+    } else if (wt == WT.JAVABIN) {
+      assertNotNull(DebugServlet.requestBody);
+    }
+  }
+
+  protected void testCollectionParameters(
+      HttpSolrClientBase baseUrlClient, HttpSolrClientBase collection1UrlClient)
+      throws IOException, SolrServerException {
+    try {
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", "collection");
+      baseUrlClient.add("collection1", doc);
+      baseUrlClient.commit("collection1");
+
+      assertEquals(
+          1,
+          baseUrlClient
+              .query("collection1", new SolrQuery("id:collection"))
+              .getResults()
+              .getNumFound());
+
+      assertEquals(
+          1, collection1UrlClient.query(new SolrQuery("id:collection")).getResults().getNumFound());
+    } finally {
+      baseUrlClient.close();
+      collection1UrlClient.close();
+    }
+  }
+
+  protected void setReqParamsOf(UpdateRequest req, String... keys) {
+    if (keys != null) {
+      for (String k : keys) {
+        req.setParam(k, k + "Value");
+      }
+    }
+  }
+
+  protected void verifyServletState(HttpSolrClientBase client, SolrRequest<?> request) {
+    // check query String
+    Iterator<String> paramNames = request.getParams().getParameterNamesIterator();
+    while (paramNames.hasNext()) {
+      String name = paramNames.next();
+      String[] values = request.getParams().getParams(name);
+      if (values != null) {
+        for (String value : values) {
+          boolean shouldBeInQueryString =
+              client.getUrlParamNames().contains(name)
+                  || (request.getQueryParams() != null && request.getQueryParams().contains(name));
+          assertEquals(
+              shouldBeInQueryString, DebugServlet.queryString.contains(name + "=" + value));
+          // in either case, it should be in the parameters
+          assertNotNull(DebugServlet.parameters.get(name));
+          assertEquals(1, DebugServlet.parameters.get(name).length);
+          assertEquals(value, DebugServlet.parameters.get(name)[0]);
+        }
+      }
+    }
+  }
+
+  protected void testQueryString() throws Exception {
+    final String clientUrl = getBaseUrl() + DEBUG_SERVLET_PATH;
+    UpdateRequest req = new UpdateRequest();
+
+    try (HttpSolrClientBase client =
+        builder(clientUrl, DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)
+            .withDefaultCollection(DEFAULT_CORE)
+            .withTheseParamNamesInTheUrl(Set.of("serverOnly"))
+            .build()) {
+      // test without request query params
+      DebugServlet.clear();
+      setReqParamsOf(req, "serverOnly", "notServer");
+
+      try {
+        client.request(req);
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+      }
+      verifyServletState(client, req);
+
+      // test without server query params
+      DebugServlet.clear();
+    }
+    try (HttpSolrClientBase client =
+        builder(clientUrl, DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)
+            .withTheseParamNamesInTheUrl(Set.of())
+            .build()) {
+      req = new UpdateRequest();
+      req.setQueryParams(Set.of("requestOnly"));
+      setReqParamsOf(req, "requestOnly", "notRequest");
+      try {
+        client.request(req);
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+      }
+      verifyServletState(client, req);
+
+      // test with both request and server query params
+      DebugServlet.clear();
+    }
+    try (HttpSolrClientBase client =
+        builder(clientUrl, DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)
+            .withTheseParamNamesInTheUrl(Set.of("serverOnly", "both"))
+            .build()) {
+      req = new UpdateRequest();
+      req.setQueryParams(Set.of("requestOnly", "both"));
+      setReqParamsOf(req, "serverOnly", "requestOnly", "both", "neither");
+      try {
+        client.request(req);
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+      }
+      verifyServletState(client, req);
+    }
+    try (HttpSolrClientBase client =
+        builder(clientUrl, DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)
+            .withTheseParamNamesInTheUrl(Set.of("serverOnly", "both"))
+            .build()) {
+
+      // test with both request and server query params with single stream
+      DebugServlet.clear();
+      req = new UpdateRequest();
+      req.add(new SolrInputDocument());
+      req.setQueryParams(Set.of("requestOnly", "both"));
+      setReqParamsOf(req, "serverOnly", "requestOnly", "both", "neither");
+      try {
+        client.request(req);
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
+      }
+      // NOTE: single stream requests send all the params
+      // as part of the query string.  So add "neither" to the request
+      // so it passes the verification step.
+      req.setQueryParams(Set.of("requestOnly", "both", "neither"));
+      verifyServletState(client, req);
+    }
+  }
+
+  protected void testGetRawStream(HttpSolrClientBase client) throws Exception {
+    DebugServlet.clear();
+    final var req = new QueryRequest(params("q", "*:*"));
+    req.setResponseParser(new InputStreamResponseParser("xml"));
+    final var rsp = req.process(client);
+    Object stream = rsp.getResponse().get("stream");
+    assertNotNull(stream);
+    MatcherAssert.assertThat(stream, instanceOf(InputStream.class));
+    InputStream is = (InputStream) stream;
+    assertNotNull(is.readAllBytes()); // throws IOException if closed
+    org.apache.solr.common.util.IOUtils.closeQuietly((InputStream) stream);
+  }
+
+  protected void testSetCredentialsExplicitly(HttpSolrClientBase client) {
+    QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
+    try {
+      ignoreException("Error from server");
+      client.request(r);
+    } catch (Exception e) {
+      // expected
+    }
+    unIgnoreException("Error from server");
+    assertTrue(DebugServlet.headers.size() > 0);
+    String authorizationHeader = DebugServlet.headers.get("authorization");
+    assertNotNull(
+        "No authorization information in headers found. Headers: " + DebugServlet.headers,
+        authorizationHeader);
+    assertEquals(
+        "Basic "
+            + Base64.getEncoder().encodeToString("foo:explicit".getBytes(StandardCharsets.UTF_8)),
+        authorizationHeader);
+  }
+
+  protected void testPerRequestCredentials(HttpSolrClientBase client) {
+    QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
+    r.setBasicAuthCredentials("foo3", "per-request");
+    try {
+      ignoreException("Error from server");
+      client.request(r);
+    } catch (Exception e) {
+      // expected
+    }
+    unIgnoreException("Error from server");
+    assertTrue(DebugServlet.headers.size() > 0);
+    String authorizationHeader = DebugServlet.headers.get("authorization");
+    assertNotNull(
+        "No authorization information in headers found. Headers: " + DebugServlet.headers,
+        authorizationHeader);
+    assertEquals(
+        "Basic "
+            + Base64.getEncoder()
+                .encodeToString("foo3:per-request".getBytes(StandardCharsets.UTF_8)),
+        authorizationHeader);
+  }
+
+  protected void testNoCredentials(HttpSolrClientBase client) {
+    QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
+    try {
+      ignoreException("Error from server");
+      client.request(r);
+    } catch (Exception e) {
+      // expected
+    }
+    unIgnoreException("Error from server");
+    assertFalse(
+        "Expecting no authorization header but got: " + DebugServlet.headers,
+        DebugServlet.headers.containsKey("authorization"));
+  }
+
+  protected void testUseOptionalCredentials(HttpSolrClientBase client) {
+    QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
+    try {
+      ignoreException("Error from server");
+      client.request(r);
+    } catch (Exception e) {
+      // expected
+    }
+    unIgnoreException("Error from server");
+    assertTrue(DebugServlet.headers.size() > 0);
+    String authorizationHeader = DebugServlet.headers.get("authorization");
+    assertNotNull(
+        "No authorization information in headers found. Headers: " + DebugServlet.headers,
+        authorizationHeader);
+    assertEquals(
+        "Basic "
+            + Base64.getEncoder().encodeToString("foo:expli:cit".getBytes(StandardCharsets.UTF_8)),
+        authorizationHeader);
+  }
+
+  protected void testUseOptionalCredentialsWithNull(HttpSolrClientBase client) {
+    // username foo, password with embedded colon separator is "expli:cit".
+    QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
+    try {
+      ignoreException("Error from server");
+      client.request(r);
+    } catch (Exception e) {
+      // expected
+    }
+    unIgnoreException("Error from server");
+    assertTrue(DebugServlet.headers.size() > 0);
+    String authorizationHeader = DebugServlet.headers.get("authorization");
+    assertNull(
+        "No authorization headers expected. Headers: " + DebugServlet.headers, authorizationHeader);
+  }
+}
diff --git a/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java b/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java
index 434ad9a6e7a..4b0c7c47529 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java
@@ -249,6 +249,21 @@ public class SSLTestConfig {
     };
   }
 
+  public KeyStore defaultKeyStore() {
+    if (keyStore == null) {
+      return null;
+    }
+    return buildKeyStore(keyStore, TEST_PASSWORD);
+  }
+
+  public String defaultKeyStorePassword() {
+    return TEST_PASSWORD;
+  }
+
+  public SecureRandom notSecureSecureRandom() {
+    return NotSecurePseudoRandom.INSTANCE;
+  }
+
   /** Constructs a KeyStore using the specified filename and password */
   private static KeyStore buildKeyStore(Resource resource, String password) {
     try {