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

[solr] branch main updated: SOLR-16623: new SolrJettyTestRule (#1374)

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

dsmiley 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 7a88bd84c19 SOLR-16623: new SolrJettyTestRule (#1374)
7a88bd84c19 is described below

commit 7a88bd84c19b4b2bc0cd08827d7d4fad3ed15518
Author: Joshua Ouma <jo...@gmail.com>
AuthorDate: Sun Aug 27 06:29:01 2023 +0300

    SOLR-16623: new SolrJettyTestRule (#1374)
    
    Introduce SolrJettyTestRule, a new subclass of SolrClientTestRule for tests requiring Jetty or HTTP.
    Perhaps most tests using JettySolrRunner could convert to this eventually.
    
    Only converted SolrJettyTestBase, (thus the many tests using it) which should be considered a deprecated test utility now. Don’t need a base test class just to use Jetty; you simply need to use SolrJettyTestRule. Changed many of these subclasses in trivial ways like to reduce Jetty references.
    
    Moved SolrJettyTestBase.legacyExampleCollection1SolrHome to SolrTestCaseJ4 because wasn’t Jetty related; was used by non-Jetty tests.
    
    
    Co-authored-by: David Smiley <ds...@salesforce.com>
---
 solr/CHANGES.txt                                   |   2 +
 .../org/apache/solr/TestSolrCoreProperties.java    |   7 +-
 .../test/org/apache/solr/TestTolerantSearch.java   |  10 +-
 .../src/test/org/apache/solr/cloud/ZkCLITest.java  |   3 +-
 .../apache/solr/core/TestConfigSetImmutable.java   |   7 +-
 .../apache/solr/core/TestSolrConfigHandler.java    |  11 +-
 .../org/apache/solr/handler/TestHttpRequestId.java |   2 +-
 .../component/DistributedDebugComponentTest.java   |  10 +-
 .../org/apache/solr/metrics/JvmMetricsTest.java    |   4 +-
 .../apache/solr/request/TestRemoteStreaming.java   |  25 +---
 .../org/apache/solr/request/TestStreamBody.java    |  12 +-
 .../org/apache/solr/rest/SolrRestletTestBase.java  |   3 +
 .../apache/solr/rest/schema/TestBulkSchemaAPI.java |   9 +-
 .../analysis/TestManagedStopFilterFactory.java     |   5 +-
 .../analysis/TestManagedSynonymFilterFactory.java  |   5 +-
 .../TestManagedSynonymGraphFilterFactory.java      |   5 +-
 .../solr/schema/TestUseDocValuesAsStored2.java     |   7 +-
 .../apache/solr/servlet/CacheHeaderTestBase.java   |   4 +-
 .../apache/solr/servlet/ResponseHeaderTest.java    |   2 +-
 .../test/org/apache/solr/ltr/TestRerankBase.java   |   9 +-
 .../store/rest/TestModelManagerPersistence.java    |  12 +-
 .../solr/handler/sql/TestSQLHandlerNonCloud.java   |   2 +-
 .../scraper/SolrStandaloneScraperTest.java         |   4 -
 .../solr/common/cloud/TestZkConfigSetService.java  |   3 +-
 .../client/solrj/SolrExampleBinaryHttp2Test.java   |   2 +-
 .../solr/client/solrj/SolrExampleBinaryTest.java   |   2 +-
 .../solr/client/solrj/SolrExampleCborTest.java     |   2 +-
 .../apache/solr/client/solrj/SolrExampleTests.java |   4 +-
 .../solr/client/solrj/SolrExampleTestsBase.java    |  32 +++++
 .../solr/client/solrj/SolrExampleXMLTest.java      |   2 +-
 .../client/solrj/SolrSchemalessExampleTest.java    |   8 +-
 .../apache/solr/client/solrj/TestBatchUpdate.java  |   6 +-
 .../solr/client/solrj/TestSolrJErrorHandling.java  |   8 +-
 .../client/solrj/embedded/JettyWebappTest.java     |   3 +-
 .../solrj/embedded/SolrExampleJettyTest.java       |   4 +-
 .../SolrExampleStreamingBinaryHttp2Test.java       |   2 +-
 .../embedded/SolrExampleStreamingBinaryTest.java   |   2 +-
 .../embedded/SolrExampleStreamingHttp2Test.java    |   4 +-
 .../solrj/embedded/SolrExampleStreamingTest.java   |   4 +-
 .../solrj/embedded/SolrExampleXMLHttp2Test.java    |   2 +-
 .../client/solrj/impl/BasicHttpSolrClientTest.java |  39 +++---
 ...oncurrentUpdateHttp2SolrClientBadInputTest.java |   4 +-
 .../impl/ConcurrentUpdateHttp2SolrClientTest.java  |  13 +-
 .../ConcurrentUpdateSolrClientBadInputTest.java    |   4 +-
 .../solrj/impl/ConcurrentUpdateSolrClientTest.java |  10 +-
 .../impl/Http2SolrClientCompatibilityTest.java     |  16 +--
 .../solrj/impl/Http2SolrClientProxyTest.java       |  51 +++-----
 .../client/solrj/impl/Http2SolrClientTest.java     |  51 ++++----
 .../solrj/impl/HttpSolrClientBadInputTest.java     |   5 +-
 .../solrj/impl/HttpSolrClientConPoolTest.java      |  30 ++---
 .../impl/HttpSolrClientSSLAuthConPoolTest.java     |   2 +-
 .../solrj/impl/LBHttpSolrClientBadInputTest.java   |   4 +-
 .../solr/client/solrj/request/SchemaTest.java      |   5 +-
 .../solrj/response/NoOpResponseParserTest.java     |   2 +-
 .../solrj/response/TestSuggesterResponse.java      |  14 +--
 .../java/org/apache/solr/SolrJettyTestBase.java    | 123 +++++--------------
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   |  41 +++++++
 .../solr/util/EmbeddedSolrServerTestRule.java      |   4 +-
 .../java/org/apache/solr/util/RestTestBase.java    |   3 +-
 .../org/apache/solr/util/SolrClientTestRule.java   |  16 ++-
 .../org/apache/solr/util/SolrJettyTestRule.java    | 134 +++++++++++++++++++++
 61 files changed, 435 insertions(+), 386 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 8c903cc364d..9f2ad6e9e46 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -163,6 +163,8 @@ Other Changes
 
 * SOLR-16915: Lower the AffinityPlacementPlugin's default minimalFreeDiskGB to 5 GB (Houston Putman)
 
+* SOLR-16623: new SolrJettyTestRule for tests needing HTTP or Jetty. (David Smiley, Joshua Ouma)
+
 ==================  9.3.0 ==================
 
 Upgrade Notes
diff --git a/solr/core/src/test/org/apache/solr/TestSolrCoreProperties.java b/solr/core/src/test/org/apache/solr/TestSolrCoreProperties.java
index b55b368cb3f..d469c532328 100644
--- a/solr/core/src/test/org/apache/solr/TestSolrCoreProperties.java
+++ b/solr/core/src/test/org/apache/solr/TestSolrCoreProperties.java
@@ -24,7 +24,6 @@ import java.util.Properties;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.embedded.JettySolrRunner;
 import org.junit.BeforeClass;
 
 /**
@@ -73,12 +72,8 @@ public class TestSolrCoreProperties extends SolrJettyTestBase {
     if (System.getProperty("solr.data.dir") == null) {
       nodeProperties.setProperty("solr.data.dir", createTempDir().toFile().getCanonicalPath());
     }
-    jetty =
-        new JettySolrRunner(
-            homeDir.toAbsolutePath().toString(), nodeProperties, buildJettyConfig());
 
-    jetty.start();
-    port = jetty.getLocalPort();
+    solrClientTestRule.startSolr(homeDir, nodeProperties, buildJettyConfig());
 
     // createJetty(homeDir.getAbsolutePath(), null, null);
   }
diff --git a/solr/core/src/test/org/apache/solr/TestTolerantSearch.java b/solr/core/src/test/org/apache/solr/TestTolerantSearch.java
index a28fc90ece1..29658475281 100644
--- a/solr/core/src/test/org/apache/solr/TestTolerantSearch.java
+++ b/solr/core/src/test/org/apache/solr/TestTolerantSearch.java
@@ -62,12 +62,12 @@ public class TestTolerantSearch extends SolrJettyTestBase {
     systemSetPropertySolrDisableUrlAllowList("true");
     File solrHome = createSolrHome();
     createAndStartJetty(solrHome.getAbsolutePath());
-    String url = jetty.getBaseUrl().toString();
+    String url = getBaseUrl();
     collection1 = getHttpSolrClient(url + "/collection1");
     collection2 = getHttpSolrClient(url + "/collection2");
 
-    String urlCollection1 = jetty.getBaseUrl().toString() + "/" + "collection1";
-    String urlCollection2 = jetty.getBaseUrl().toString() + "/" + "collection2";
+    String urlCollection1 = getBaseUrl() + "/" + "collection1";
+    String urlCollection2 = getBaseUrl() + "/" + "collection2";
     shard1 = urlCollection1.replaceAll("https?://", "");
     shard2 = urlCollection2.replaceAll("https?://", "");
 
@@ -109,10 +109,6 @@ public class TestTolerantSearch extends SolrJettyTestBase {
       collection2.close();
       collection2 = null;
     }
-    if (null != jetty) {
-      jetty.stop();
-      jetty = null;
-    }
     resetExceptionIgnores();
     systemClearPropertySolrDisableUrlAllowList();
   }
diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java b/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
index 90a0f1a4580..c84d180a6b9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
@@ -30,7 +30,6 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.filefilter.RegexFileFilter;
 import org.apache.commons.io.filefilter.TrueFileFilter;
-import org.apache.solr.SolrJettyTestBase;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterProperties;
@@ -83,7 +82,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
       log.info("####SETUP_START {}", getTestName());
     }
 
-    String exampleHome = SolrJettyTestBase.legacyExampleCollection1SolrHome();
+    String exampleHome = legacyExampleCollection1SolrHome();
 
     Path tmpDir = createTempDir();
     solrHome = exampleHome;
diff --git a/solr/core/src/test/org/apache/solr/core/TestConfigSetImmutable.java b/solr/core/src/test/org/apache/solr/core/TestConfigSetImmutable.java
index 682469f85f3..a3282439ac3 100644
--- a/solr/core/src/test/org/apache/solr/core/TestConfigSetImmutable.java
+++ b/solr/core/src/test/org/apache/solr/core/TestConfigSetImmutable.java
@@ -66,11 +66,8 @@ public class TestConfigSetImmutable extends RestTestBase {
 
   @After
   public void after() throws Exception {
-    if (jetty != null) {
-      jetty.stop();
-      jetty = null;
-    }
-    client = null;
+    solrClientTestRule.reset();
+
     if (restTestHarness != null) {
       restTestHarness.close();
     }
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
index 44ad982c714..515f28c86eb 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrConfigHandler.java
@@ -133,17 +133,14 @@ public class TestSolrConfigHandler extends RestTestBase {
     if (random().nextBoolean()) {
       log.info("These tests are run with V2 API");
       restTestHarness.setServerProvider(
-          () -> jetty.getBaseUrl().toString() + "/____v2/cores/" + DEFAULT_TEST_CORENAME);
+          () -> getBaseUrl() + "/____v2/cores/" + DEFAULT_TEST_CORENAME);
     }
   }
 
   @After
   public void after() throws Exception {
-    if (jetty != null) {
-      jetty.stop();
-      jetty = null;
-    }
-    client = null;
+    solrClientTestRule.reset();
+
     if (restTestHarness != null) {
       restTestHarness.close();
     }
@@ -973,7 +970,7 @@ public class TestSolrConfigHandler extends RestTestBase {
         TIMEOUT_S);
     RESTfulServerProvider oldProvider = restTestHarness.getServerProvider();
     restTestHarness.setServerProvider(
-        () -> jetty.getBaseUrl().toString() + "/____v2/cores/" + DEFAULT_TEST_CORENAME);
+        () -> getBaseUrl() + "/____v2/cores/" + DEFAULT_TEST_CORENAME);
 
     Map<?, ?> rsp =
         TestSolrConfigHandler.testForResponseElement(
diff --git a/solr/core/src/test/org/apache/solr/handler/TestHttpRequestId.java b/solr/core/src/test/org/apache/solr/handler/TestHttpRequestId.java
index 7e8a15004d7..b97b9f9a143 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestHttpRequestId.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestHttpRequestId.java
@@ -105,7 +105,7 @@ public class TestHttpRequestId extends SolrJettyTestBase {
               new SolrNamedThreadFactory("httpShardExecutor"),
               false);
       try (Http2SolrClient client =
-          new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + collection)
+          new Http2SolrClient.Builder(getBaseUrl() + collection)
               .withExecutor(commExecutor)
               .build()) {
         MDC.put(key, value);
diff --git a/solr/core/src/test/org/apache/solr/handler/component/DistributedDebugComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/DistributedDebugComponentTest.java
index 58cbe43717d..374cee78888 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/DistributedDebugComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/DistributedDebugComponentTest.java
@@ -61,13 +61,13 @@ public class DistributedDebugComponentTest extends SolrJettyTestBase {
     systemSetPropertySolrDisableUrlAllowList("true");
     File solrHome = createSolrHome();
     createAndStartJetty(solrHome.getAbsolutePath());
-    String url = jetty.getBaseUrl().toString();
+    String url = getBaseUrl();
 
     collection1 = getHttpSolrClient(url + "/collection1");
     collection2 = getHttpSolrClient(url + "/collection2");
 
-    String urlCollection1 = jetty.getBaseUrl().toString() + "/" + "collection1";
-    String urlCollection2 = jetty.getBaseUrl().toString() + "/" + "collection2";
+    String urlCollection1 = getBaseUrl() + "/" + "collection1";
+    String urlCollection2 = getBaseUrl() + "/" + "collection2";
     shard1 = urlCollection1.replaceAll("https?://", "");
     shard2 = urlCollection2.replaceAll("https?://", "");
 
@@ -101,10 +101,6 @@ public class DistributedDebugComponentTest extends SolrJettyTestBase {
       collection2.close();
       collection2 = null;
     }
-    if (null != jetty) {
-      jetty.stop();
-      jetty = null;
-    }
     resetExceptionIgnores();
     systemClearPropertySolrDisableUrlAllowList();
   }
diff --git a/solr/core/src/test/org/apache/solr/metrics/JvmMetricsTest.java b/solr/core/src/test/org/apache/solr/metrics/JvmMetricsTest.java
index 970818a27d4..b7dbd43a611 100644
--- a/solr/core/src/test/org/apache/solr/metrics/JvmMetricsTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/JvmMetricsTest.java
@@ -89,7 +89,7 @@ public class JvmMetricsTest extends SolrJettyTestBase {
       // make sure it's set
       System.setProperty("basicauth", "foo:bar");
     }
-    SolrMetricManager metricManager = jetty.getCoreContainer().getMetricManager();
+    SolrMetricManager metricManager = getJetty().getCoreContainer().getMetricManager();
     Map<String, Metric> metrics = metricManager.registry("solr.jvm").getMetrics();
     Metric metric = metrics.get("system.properties");
     assertNotNull(metrics.toString(), metric);
@@ -125,7 +125,7 @@ public class JvmMetricsTest extends SolrJettyTestBase {
 
   @Test
   public void testSetupJvmMetrics() {
-    SolrMetricManager metricManager = jetty.getCoreContainer().getMetricManager();
+    SolrMetricManager metricManager = getJetty().getCoreContainer().getMetricManager();
     Map<String, Metric> metrics = metricManager.registry("solr.jvm").getMetrics();
     assertTrue(metrics.size() > 0);
     assertTrue(
diff --git a/solr/core/src/test/org/apache/solr/request/TestRemoteStreaming.java b/solr/core/src/test/org/apache/solr/request/TestRemoteStreaming.java
index e36fbb1969c..486b3396afd 100644
--- a/solr/core/src/test/org/apache/solr/request/TestRemoteStreaming.java
+++ b/solr/core/src/test/org/apache/solr/request/TestRemoteStreaming.java
@@ -34,7 +34,6 @@ import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.embedded.JettySolrRunner;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -43,8 +42,6 @@ import org.junit.Test;
 @SuppressSSL // does not yet work with ssl yet - uses raw java.net.URL API rather than HttpClient
 public class TestRemoteStreaming extends SolrJettyTestBase {
 
-  private static JettySolrRunner jettySolrRunner;
-
   @BeforeClass
   public static void beforeTest() throws Exception {
     System.setProperty("solr.enableRemoteStreaming", "true");
@@ -52,7 +49,7 @@ public class TestRemoteStreaming extends SolrJettyTestBase {
     // this one has handleSelect=true which a test here needs
     File solrHomeDirectory = createTempDir(LuceneTestCase.getTestClass().getSimpleName()).toFile();
     setupJettyTestHome(solrHomeDirectory, "collection1");
-    jettySolrRunner = createAndStartJetty(solrHomeDirectory.getAbsolutePath());
+    createAndStartJetty(solrHomeDirectory.getAbsolutePath());
   }
 
   @Before
@@ -75,16 +72,10 @@ public class TestRemoteStreaming extends SolrJettyTestBase {
 
   @Test
   public void testStreamUrl() throws Exception {
-    String streamUrl =
-        jettySolrRunner.getBaseUrl().toString()
-            + "/"
-            + DEFAULT_TEST_COLLECTION_NAME
-            + "/select?q=*:*&fl=id&wt=csv";
+    String streamUrl = getCoreUrl() + "/select?q=*:*&fl=id&wt=csv";
 
     String getUrl =
-        jettySolrRunner.getBaseUrl().toString()
-            + "/"
-            + DEFAULT_TEST_COLLECTION_NAME
+        getCoreUrl()
             + "/debug/dump?wt=xml&stream.url="
             + URLEncoder.encode(streamUrl, StandardCharsets.UTF_8);
     String content = attemptHttpGet(getUrl);
@@ -109,18 +100,14 @@ public class TestRemoteStreaming extends SolrJettyTestBase {
     SolrQuery query = new SolrQuery();
     query.setQuery("*:*"); // for anything
     query.add("stream.url", makeDeleteAllUrl());
-    try (SolrClient solrClient = createNewSolrClient()) {
-      SolrException se = expectThrows(SolrException.class, () -> solrClient.query(query));
-      assertSame(ErrorCode.BAD_REQUEST, ErrorCode.getErrorCode(se.code()));
-    }
+    SolrException se = expectThrows(SolrException.class, () -> getSolrClient().query(query));
+    assertSame(ErrorCode.BAD_REQUEST, ErrorCode.getErrorCode(se.code()));
   }
 
   /** Compose an HTTP GET url that will delete all the data. */
   private String makeDeleteAllUrl() {
     String deleteQuery = "<delete><query>*:*</query></delete>";
-    return jettySolrRunner.getBaseUrl().toString()
-        + "/"
-        + DEFAULT_TEST_COLLECTION_NAME
+    return getCoreUrl()
         + "/update?commit=true&stream.body="
         + URLEncoder.encode(deleteQuery, StandardCharsets.UTF_8);
   }
diff --git a/solr/core/src/test/org/apache/solr/request/TestStreamBody.java b/solr/core/src/test/org/apache/solr/request/TestStreamBody.java
index e495353b613..197fd2d5986 100644
--- a/solr/core/src/test/org/apache/solr/request/TestStreamBody.java
+++ b/solr/core/src/test/org/apache/solr/request/TestStreamBody.java
@@ -56,20 +56,14 @@ public class TestStreamBody extends RestTestBase {
     if (random().nextBoolean()) {
       log.info("These tests are run with V2 API");
       restTestHarness.setServerProvider(
-          () -> jetty.getBaseUrl().toString() + "/____v2/cores/" + DEFAULT_TEST_CORENAME);
+          () -> getBaseUrl() + "/____v2/cores/" + DEFAULT_TEST_CORENAME);
     }
   }
 
   @After
   public void after() throws Exception {
-    if (jetty != null) {
-      jetty.stop();
-      jetty = null;
-    }
-    if (client != null) {
-      client.close();
-      client = null;
-    }
+    solrClientTestRule.reset();
+
     if (restTestHarness != null) {
       restTestHarness.close();
       restTestHarness = null;
diff --git a/solr/core/src/test/org/apache/solr/rest/SolrRestletTestBase.java b/solr/core/src/test/org/apache/solr/rest/SolrRestletTestBase.java
index 480c2885708..59d2a35d88c 100644
--- a/solr/core/src/test/org/apache/solr/rest/SolrRestletTestBase.java
+++ b/solr/core/src/test/org/apache/solr/rest/SolrRestletTestBase.java
@@ -46,6 +46,9 @@ public abstract class SolrRestletTestBase extends RestTestBase {
     System.setProperty("coreRootDirectory", coresDir.toString());
     System.setProperty("configSetBaseDir", TEST_HOME());
 
+    System.setProperty("solr.test.sys.prop1", "propone");
+    System.setProperty("solr.test.sys.prop2", "proptwo");
+
     final SortedMap<ServletHolder, String> extraServlets = new TreeMap<>();
 
     Properties props = new Properties();
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
index e6db67578b0..98a65a3ebae 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java
@@ -78,7 +78,7 @@ public class TestBulkSchemaAPI extends RestTestBase {
           new RESTfulServerProvider() {
             @Override
             public String getBaseURL() {
-              return jetty.getBaseUrl().toString() + "/____v2/cores/" + DEFAULT_TEST_CORENAME;
+              return getBaseUrl() + "/____v2/cores/" + DEFAULT_TEST_CORENAME;
             }
           });
     }
@@ -86,10 +86,7 @@ public class TestBulkSchemaAPI extends RestTestBase {
 
   @After
   public void after() throws Exception {
-    if (jetty != null) {
-      jetty.stop();
-      jetty = null;
-    }
+    solrClientTestRule.reset();
     if (restTestHarness != null) {
       restTestHarness.close();
     }
@@ -1459,7 +1456,7 @@ public class TestBulkSchemaAPI extends RestTestBase {
   @SuppressWarnings({"unchecked", "varargs"})
   private static <T extends Similarity> void assertFieldSimilarity(
       String fieldname, Class<T> expected, Consumer<T>... validators) {
-    CoreContainer cc = jetty.getCoreContainer();
+    CoreContainer cc = solrClientTestRule.getCoreContainer();
     try (SolrCore core = cc.getCore("collection1")) {
       SimilarityFactory simfac = core.getLatestSchema().getSimilarityFactory();
       assertNotNull(simfac);
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedStopFilterFactory.java b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedStopFilterFactory.java
index fb6633e2848..a1fff16edc0 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedStopFilterFactory.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedStopFilterFactory.java
@@ -61,10 +61,7 @@ public class TestManagedStopFilterFactory extends RestTestBase {
 
   @After
   public void after() throws Exception {
-    if (null != jetty) {
-      jetty.stop();
-      jetty = null;
-    }
+    solrClientTestRule.reset();
     System.clearProperty("managed.schema.mutable");
     System.clearProperty("enable.update.log");
 
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java
index 87cba4ae404..4738196017c 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java
@@ -59,10 +59,7 @@ public class TestManagedSynonymFilterFactory extends RestTestBase {
 
   @After
   public void after() throws Exception {
-    if (null != jetty) {
-      jetty.stop();
-      jetty = null;
-    }
+    solrClientTestRule.reset();
     if (null != tmpSolrHome) {
       PathUtils.deleteDirectory(tmpSolrHome);
       tmpSolrHome = null;
diff --git a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java
index f93482a00d0..c5363c7f251 100644
--- a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java
+++ b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymGraphFilterFactory.java
@@ -63,10 +63,7 @@ public class TestManagedSynonymGraphFilterFactory extends RestTestBase {
 
   @After
   public void after() throws Exception {
-    if (null != jetty) {
-      jetty.stop();
-      jetty = null;
-    }
+    solrClientTestRule.reset();
     if (null != tmpSolrHome) {
       PathUtils.deleteDirectory(tmpSolrHome.toPath());
     }
diff --git a/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored2.java b/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored2.java
index 1f05641a6d1..0a1c6d19dc8 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored2.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored2.java
@@ -50,11 +50,8 @@ public class TestUseDocValuesAsStored2 extends RestTestBase {
 
   @After
   public void after() throws Exception {
-    if (jetty != null) {
-      jetty.stop();
-      jetty = null;
-    }
-    client = null;
+    solrClientTestRule.reset();
+
     if (restTestHarness != null) {
       restTestHarness.close();
     }
diff --git a/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTestBase.java b/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTestBase.java
index b372b4b3c8f..5f25e8140d8 100644
--- a/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTestBase.java
+++ b/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTestBase.java
@@ -46,7 +46,7 @@ public abstract class CacheHeaderTestBase extends SolrJettyTestBase {
 
     URI uri =
         URI.create(
-            jetty.getBaseUrl().toString()
+            getBaseUrl()
                 + "/"
                 + DEFAULT_TEST_COLLECTION_NAME
                 + "/select?"
@@ -73,7 +73,7 @@ public abstract class CacheHeaderTestBase extends SolrJettyTestBase {
 
     URI uri =
         URI.create(
-            jetty.getBaseUrl()
+            getBaseUrl()
                 + "/"
                 + DEFAULT_TEST_COLLECTION_NAME
                 + "/update?"
diff --git a/solr/core/src/test/org/apache/solr/servlet/ResponseHeaderTest.java b/solr/core/src/test/org/apache/solr/servlet/ResponseHeaderTest.java
index b5d312eb021..b9e821c3acb 100644
--- a/solr/core/src/test/org/apache/solr/servlet/ResponseHeaderTest.java
+++ b/solr/core/src/test/org/apache/solr/servlet/ResponseHeaderTest.java
@@ -58,7 +58,7 @@ public class ResponseHeaderTest extends SolrJettyTestBase {
 
   @Test
   public void testHttpResponse() throws IOException {
-    URI uri = URI.create(jetty.getBaseUrl() + "/collection1/withHeaders?q=*:*");
+    URI uri = URI.create(getBaseUrl() + "/collection1/withHeaders?q=*:*");
     HttpGet httpGet = new HttpGet(uri);
     HttpResponse response = getHttpClient().execute(httpGet);
     Header[] headers = response.getAllHeaders();
diff --git a/solr/modules/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java b/solr/modules/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java
index 3089263c036..88a60efea59 100644
--- a/solr/modules/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java
+++ b/solr/modules/ltr/src/test/org/apache/solr/ltr/TestRerankBase.java
@@ -121,13 +121,13 @@ public class TestRerankBase extends RestTestBase {
   }
 
   public static ManagedFeatureStore getManagedFeatureStore() {
-    try (SolrCore core = jetty.getCoreContainer().getCore(DEFAULT_TEST_CORENAME)) {
+    try (SolrCore core = solrClientTestRule.getCoreContainer().getCore(DEFAULT_TEST_CORENAME)) {
       return ManagedFeatureStore.getManagedFeatureStore(core);
     }
   }
 
   public static ManagedModelStore getManagedModelStore() {
-    try (SolrCore core = jetty.getCoreContainer().getCore(DEFAULT_TEST_CORENAME)) {
+    try (SolrCore core = solrClientTestRule.getCoreContainer().getCore(DEFAULT_TEST_CORENAME)) {
       return ManagedModelStore.getManagedModelStore(core);
     }
   }
@@ -195,10 +195,7 @@ public class TestRerankBase extends RestTestBase {
       restTestHarness.close();
       restTestHarness = null;
     }
-    if (null != jetty) {
-      jetty.stop();
-      jetty = null;
-    }
+    solrClientTestRule.reset();
     if (null != tmpSolrHome) {
       PathUtils.deleteDirectory(tmpSolrHome);
       tmpSolrHome = null;
diff --git a/solr/modules/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java b/solr/modules/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java
index d513cfd9dc7..2adcc1f19f8 100644
--- a/solr/modules/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java
+++ b/solr/modules/ltr/src/test/org/apache/solr/ltr/store/rest/TestModelManagerPersistence.java
@@ -170,8 +170,8 @@ public class TestModelManagerPersistence extends TestRerankBase {
         "/features/[1]/name=='description'");
 
     // check persistence after restart
-    jetty.stop();
-    jetty.start();
+    getJetty().stop();
+    getJetty().start();
     assertJQ(ManagedModelStore.REST_END_POINT, "/models/[0]/name=='" + modelName + "'");
     assertJQ(
         ManagedFeatureStore.REST_END_POINT + "/" + FeatureStore.DEFAULT_FEATURE_STORE_NAME,
@@ -197,8 +197,8 @@ public class TestModelManagerPersistence extends TestRerankBase {
         "/error/msg=='Missing feature store: " + FeatureStore.DEFAULT_FEATURE_STORE_NAME + "'");
 
     // check persistence after restart
-    jetty.stop();
-    jetty.start();
+    getJetty().stop();
+    getJetty().start();
     assertJQ(ManagedModelStore.REST_END_POINT, "/models/==[]");
     assertJQ(
         ManagedFeatureStore.REST_END_POINT + "/" + FeatureStore.DEFAULT_FEATURE_STORE_NAME,
@@ -269,8 +269,8 @@ public class TestModelManagerPersistence extends TestRerankBase {
     doWrapperModelPersistenceChecks(modelName, FS_NAME, baseModelFile.getFileName().toString());
 
     // check persistence after restart
-    jetty.stop();
-    jetty.start();
+    getJetty().stop();
+    getJetty().start();
     doWrapperModelPersistenceChecks(modelName, FS_NAME, baseModelFile.getFileName().toString());
 
     // delete test settings
diff --git a/solr/modules/sql/src/test/org/apache/solr/handler/sql/TestSQLHandlerNonCloud.java b/solr/modules/sql/src/test/org/apache/solr/handler/sql/TestSQLHandlerNonCloud.java
index a0af9117b24..7dd50fcb5e7 100644
--- a/solr/modules/sql/src/test/org/apache/solr/handler/sql/TestSQLHandlerNonCloud.java
+++ b/solr/modules/sql/src/test/org/apache/solr/handler/sql/TestSQLHandlerNonCloud.java
@@ -49,7 +49,7 @@ public class TestSQLHandlerNonCloud extends SolrJettyTestBase {
   public void testSQLHandler() throws Exception {
     String sql = "select id, field_i, str_s from " + DEFAULT_TEST_COLLECTION_NAME + " limit 10";
     SolrParams sParams = params(CommonParams.QT, "/sql", "stmt", sql);
-    String url = jetty.getBaseUrl() + "/" + DEFAULT_TEST_COLLECTION_NAME;
+    String url = getBaseUrl() + "/" + DEFAULT_TEST_COLLECTION_NAME;
 
     SolrStream solrStream = new SolrStream(url, sParams);
     IOException ex = expectThrows(IOException.class, () -> getTuples(solrStream));
diff --git a/solr/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrStandaloneScraperTest.java b/solr/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrStandaloneScraperTest.java
index c254cc1e1e4..1db0142c2ea 100644
--- a/solr/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrStandaloneScraperTest.java
+++ b/solr/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrStandaloneScraperTest.java
@@ -81,10 +81,6 @@ public class SolrStandaloneScraperTest extends RestTestBase {
       executor.shutdownNow();
       executor = null;
     }
-    if (null != jetty) {
-      jetty.stop();
-      jetty = null;
-    }
     solrScraper = null;
     solrClient = null;
   }
diff --git a/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/TestZkConfigSetService.java b/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/TestZkConfigSetService.java
index 058921cc8fb..16454d5dc3e 100644
--- a/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/TestZkConfigSetService.java
+++ b/solr/solrj-zookeeper/src/test/org/apache/solr/common/cloud/TestZkConfigSetService.java
@@ -27,7 +27,6 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
-import org.apache.solr.SolrJettyTestBase;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.cloud.AbstractZkTestCase;
 import org.apache.solr.cloud.ZkConfigSetService;
@@ -240,7 +239,7 @@ public class TestZkConfigSetService extends SolrTestCaseJ4 {
   @Test
   public void testBootstrapConf() throws IOException, KeeperException, InterruptedException {
 
-    String solrHome = SolrJettyTestBase.legacyExampleCollection1SolrHome();
+    String solrHome = legacyExampleCollection1SolrHome();
 
     CoreContainer cc = new CoreContainer(Paths.get(solrHome), new Properties());
     System.setProperty("zkHost", zkServer.getZkAddress());
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryHttp2Test.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryHttp2Test.java
index ceae9494376..331104ca195 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryHttp2Test.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryHttp2Test.java
@@ -38,7 +38,7 @@ public class SolrExampleBinaryHttp2Test extends SolrExampleTests {
 
   @Override
   public SolrClient createNewSolrClient() {
-    return new Http2SolrClient.Builder(getServerUrl())
+    return new Http2SolrClient.Builder(getCoreUrl())
         .withConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)
         .withRequestWriter(new BinaryRequestWriter())
         // where the magic happens
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java
index e7c58b8ea4a..897e37c2ab0 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java
@@ -35,7 +35,7 @@ public class SolrExampleBinaryTest extends SolrExampleTests {
 
   @Override
   public SolrClient createNewSolrClient() {
-    return new HttpSolrClient.Builder(getServerUrl())
+    return new HttpSolrClient.Builder(getCoreUrl())
         .allowMultiPartPost(random().nextBoolean())
         .withRequestWriter(new BinaryRequestWriter())
         .withResponseParser(new BinaryResponseParser())
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleCborTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleCborTest.java
index c8b818094d2..9441eacfa1d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleCborTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleCborTest.java
@@ -52,7 +52,7 @@ public class SolrExampleCborTest extends SolrExampleTests {
 
   @Override
   public SolrClient createNewSolrClient() {
-    return new HttpSolrClient.Builder(getServerUrl())
+    return new HttpSolrClient.Builder(getCoreUrl())
         .allowMultiPartPost(random().nextBoolean())
         .withRequestWriter(cborRequestWriter())
         .withResponseParser(cborResponseparser())
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
index df55b78ab1e..06c0da66e05 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
@@ -435,9 +435,9 @@ public abstract class SolrExampleTests extends SolrExampleTestsBase {
     assertEquals("price:[* TO 2]", values.get(0));
     assertEquals("price:[2 TO 4]", values.get(1));
 
-    if (jetty != null) {
+    if (getJetty() != null) {
       // check system wide system handler + "/admin/info/system"
-      String url = jetty.getBaseUrl().toString();
+      String url = getBaseUrl();
       try (SolrClient adminClient = getHttpSolrClient(url)) {
         SolrQuery q = new SolrQuery();
         q.set("qt", "/admin/info/system");
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java
index 19c0c2407b4..88d454cfe52 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestsBase.java
@@ -31,9 +31,41 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.util.TimeOut;
+import org.junit.After;
 import org.junit.Test;
 
 public abstract class SolrExampleTestsBase extends SolrJettyTestBase {
+  private SolrClient client;
+
+  @After
+  public void after() {
+    if (client != null) {
+      try {
+        client.close();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    client = null;
+  }
+
+  @Override
+  public SolrClient getSolrClient() {
+    if (client == null) {
+      client = createNewSolrClient();
+    }
+    return client;
+  }
+
+  /**
+   * Create a new solr client. If createJetty was called, a http implementation will be created,
+   * otherwise an embedded implementation will be created. Subclasses should override for other
+   * options.
+   */
+  @Override
+  public SolrClient createNewSolrClient() {
+    return getHttpSolrClient(getCoreUrl());
+  }
 
   /** query the example */
   @Test
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java
index de312169ba6..faa4c176526 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java
@@ -35,7 +35,7 @@ public class SolrExampleXMLTest extends SolrExampleTests {
 
   @Override
   public SolrClient createNewSolrClient() {
-    HttpSolrClient.Builder httpSolrClientBuilder = new HttpSolrClient.Builder(getServerUrl());
+    HttpSolrClient.Builder httpSolrClientBuilder = new HttpSolrClient.Builder(getCoreUrl());
     httpSolrClientBuilder.allowMultiPartPost(random().nextBoolean());
 
     httpSolrClientBuilder
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java
index 0a365bdedc9..f188c66f299 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java
@@ -83,8 +83,7 @@ public class SolrSchemalessExampleTest extends SolrExampleTestsBase {
     // two docs, one with uniqueKey, another without it
     String json = "{\"id\":\"abc1\", \"name\": \"name1\"} {\"name\" : \"name2\"}";
     HttpClient httpClient = getHttpClient();
-    HttpPost post =
-        new HttpPost(jetty.getBaseUrl() + "/" + DEFAULT_TEST_COLLECTION_NAME + "/update/json/docs");
+    HttpPost post = new HttpPost(getCoreUrl() + "/update/json/docs");
     post.setHeader("Content-Type", "application/json");
     post.setEntity(
         new InputStreamEntity(new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8)), -1));
@@ -112,8 +111,7 @@ public class SolrSchemalessExampleTest extends SolrExampleTestsBase {
             + "{\"p.q\" : \"name\"}"
             + "{\"a&b\" : \"name\"}";
     HttpClient httpClient = getHttpClient();
-    HttpPost post =
-        new HttpPost(jetty.getBaseUrl() + "/" + DEFAULT_TEST_COLLECTION_NAME + "/update/json/docs");
+    HttpPost post = new HttpPost(getCoreUrl() + "/update/json/docs");
     post.setHeader("Content-Type", "application/json");
     post.setEntity(
         new InputStreamEntity(new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8)), -1));
@@ -132,7 +130,7 @@ public class SolrSchemalessExampleTest extends SolrExampleTestsBase {
 
   @Override
   public SolrClient createNewSolrClient() {
-    HttpSolrClient.Builder httpSolrClientBuilder = new HttpSolrClient.Builder(getServerUrl());
+    HttpSolrClient.Builder httpSolrClientBuilder = new HttpSolrClient.Builder(getCoreUrl());
     if (random().nextBoolean()) {
       httpSolrClientBuilder
           .withRequestWriter(new BinaryRequestWriter())
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/TestBatchUpdate.java b/solr/solrj/src/test/org/apache/solr/client/solrj/TestBatchUpdate.java
index 5dd599fcd45..68d70c8ca44 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/TestBatchUpdate.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/TestBatchUpdate.java
@@ -47,7 +47,7 @@ public class TestBatchUpdate extends SolrJettyTestBase {
   @Test
   public void testWithXml() throws Exception {
     try (SolrClient client =
-        new HttpSolrClient.Builder(getServerUrl()).withRequestWriter(new RequestWriter()).build()) {
+        new HttpSolrClient.Builder(getCoreUrl()).withRequestWriter(new RequestWriter()).build()) {
       client.deleteByQuery("*:*"); // delete everything!
       doIt(client);
     }
@@ -56,7 +56,7 @@ public class TestBatchUpdate extends SolrJettyTestBase {
   @Test
   public void testWithBinary() throws Exception {
     try (SolrClient client =
-        new HttpSolrClient.Builder(getServerUrl())
+        new HttpSolrClient.Builder(getCoreUrl())
             .withRequestWriter(new BinaryRequestWriter())
             .build()) {
       client.deleteByQuery("*:*"); // delete everything!
@@ -67,7 +67,7 @@ public class TestBatchUpdate extends SolrJettyTestBase {
   @Test
   public void testWithBinaryBean() throws Exception {
     try (SolrClient client =
-        new HttpSolrClient.Builder(getServerUrl())
+        new HttpSolrClient.Builder(getCoreUrl())
             .withRequestWriter(new BinaryRequestWriter())
             .build()) {
       client.deleteByQuery("*:*"); // delete everything!
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/TestSolrJErrorHandling.java b/solr/solrj/src/test/org/apache/solr/client/solrj/TestSolrJErrorHandling.java
index a2feae3f5ab..789d39d516d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/TestSolrJErrorHandling.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/TestSolrJErrorHandling.java
@@ -102,7 +102,7 @@ public class TestSolrJErrorHandling extends SolrJettyTestBase {
   @Test
   public void testWithXml() throws Exception {
     try (SolrClient client =
-        new HttpSolrClient.Builder(getServerUrl()).withRequestWriter(new RequestWriter()).build()) {
+        new HttpSolrClient.Builder(getCoreUrl()).withRequestWriter(new RequestWriter()).build()) {
 
       client.deleteByQuery("*:*"); // delete everything!
       doIt(client);
@@ -112,7 +112,7 @@ public class TestSolrJErrorHandling extends SolrJettyTestBase {
   @Test
   public void testWithBinary() throws Exception {
     try (SolrClient client =
-        new HttpSolrClient.Builder(getServerUrl())
+        new HttpSolrClient.Builder(getCoreUrl())
             .withRequestWriter(new BinaryRequestWriter())
             .build()) {
       client.deleteByQuery("*:*"); // delete everything!
@@ -274,7 +274,7 @@ public class TestSolrJErrorHandling extends SolrJettyTestBase {
     // sometimes succeeds with this size, but larger can cause OOM from command line
     String bodyString = getJsonDocs(200000);
 
-    String urlString = jetty.getBaseUrl() + "/" + DEFAULT_TEST_COLLECTION_NAME + "/update";
+    String urlString = getCoreUrl() + "/update";
 
     HttpURLConnection conn = null;
     URL url = new URL(urlString);
@@ -327,7 +327,7 @@ public class TestSolrJErrorHandling extends SolrJettyTestBase {
   public void testRawSocket() throws Exception {
 
     String hostName = "127.0.0.1";
-    int port = jetty.getLocalPort();
+    int port = getJetty().getLocalPort();
 
     try (Socket socket = new Socket(hostName, port);
         OutputStream out = new BufferedOutputStream(socket.getOutputStream());
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java
index ec631da7081..61968db2e4d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java
@@ -27,7 +27,6 @@ import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpRequestBase;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClients;
-import org.apache.solr.SolrJettyTestBase;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.util.ExternalPaths;
@@ -49,7 +48,7 @@ public class JettyWebappTest extends SolrTestCaseJ4 {
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    System.setProperty("solr.solr.home", SolrJettyTestBase.legacyExampleCollection1SolrHome());
+    System.setProperty("solr.solr.home", legacyExampleCollection1SolrHome());
     System.setProperty("tests.shardhandler.randomSeed", Long.toString(random().nextLong()));
     System.setProperty("solr.tests.doContainerStreamCloseAssert", "false");
 
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
index 310054881c1..11a593d82c3 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
@@ -73,8 +73,8 @@ public class SolrExampleJettyTest extends SolrExampleTests {
 
       // two docs, one with uniqueKey, another without it
       String json = "{\"id\":\"abc1\", \"name\": \"name1\"} {\"name\" : \"name2\"}";
-      HttpClient httpClient = getHttpClient(getServerUrl());
-      HttpPost post = new HttpPost(getRandomizedUpdateUri(getServerUrl()));
+      HttpClient httpClient = getHttpClient(getCoreUrl());
+      HttpPost post = new HttpPost(getRandomizedUpdateUri(getCoreUrl()));
       post.setHeader("Content-Type", "application/json");
       post.setEntity(
           new InputStreamEntity(
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingBinaryHttp2Test.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingBinaryHttp2Test.java
index 32f49d2c69c..cbab498a9ac 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingBinaryHttp2Test.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingBinaryHttp2Test.java
@@ -37,7 +37,7 @@ public class SolrExampleStreamingBinaryHttp2Test extends SolrExampleStreamingHtt
 
   @Override
   public SolrClient createNewSolrClient() {
-    String url = jetty.getBaseUrl().toString() + "/collection1";
+    String url = getCoreUrl();
     // smaller queue size hits locks more often
     Http2SolrClient solrClient =
         new Http2SolrClient.Builder()
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingBinaryTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingBinaryTest.java
index 77310ff3055..ec326f61858 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingBinaryTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingBinaryTest.java
@@ -36,7 +36,7 @@ public class SolrExampleStreamingBinaryTest extends SolrExampleStreamingTest {
   public SolrClient createNewSolrClient() {
 
     SolrClient client =
-        new ErrorTrackingConcurrentUpdateSolrClient.Builder(getServerUrl())
+        new ErrorTrackingConcurrentUpdateSolrClient.Builder(getCoreUrl())
             .withQueueSize(2)
             .withThreadCount(5)
             .withResponseParser(new BinaryResponseParser())
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingHttp2Test.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingHttp2Test.java
index f7ea0d83961..1584fc87cf1 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingHttp2Test.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingHttp2Test.java
@@ -45,7 +45,7 @@ public class SolrExampleStreamingHttp2Test extends SolrExampleTests {
 
   @Override
   public SolrClient createNewSolrClient() {
-    String url = jetty.getBaseUrl().toString() + "/collection1";
+    String url = getCoreUrl();
     // smaller queue size hits locks more often
     Http2SolrClient solrClient =
         new Http2SolrClient.Builder()
@@ -63,7 +63,7 @@ public class SolrExampleStreamingHttp2Test extends SolrExampleTests {
   public void testWaitOptions() throws Exception {
     // SOLR-3903
     final List<Throwable> failures = new ArrayList<>();
-    final String serverUrl = jetty.getBaseUrl().toString() + "/collection1";
+    final String serverUrl = getCoreUrl();
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         ConcurrentUpdateHttp2SolrClient concurrentClient =
             new FailureRecordingConcurrentUpdateSolrClient.Builder(serverUrl, http2Client)
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java
index 2e06cdadfed..6aa438e0339 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java
@@ -44,7 +44,7 @@ public class SolrExampleStreamingTest extends SolrExampleTests {
   @Override
   public SolrClient createNewSolrClient() {
     // smaller queue size hits locks more often
-    return new ErrorTrackingConcurrentUpdateSolrClient.Builder(getServerUrl())
+    return new ErrorTrackingConcurrentUpdateSolrClient.Builder(getCoreUrl())
         .withQueueSize(2)
         .withThreadCount(5)
         .withResponseParser(new XMLResponseParser())
@@ -56,7 +56,7 @@ public class SolrExampleStreamingTest extends SolrExampleTests {
     // SOLR-3903
     // TODO these failures are not the same as recorded by the client
     final List<Throwable> failures = new ArrayList<>();
-    final String serverUrl = jetty.getBaseUrl().toString() + "/collection1";
+    final String serverUrl = getCoreUrl();
     try (ConcurrentUpdateSolrClient concurrentClient =
         new FailureRecordingConcurrentUpdateSolrClient.Builder(serverUrl)
             .withQueueSize(2)
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleXMLHttp2Test.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleXMLHttp2Test.java
index d495b6b0a0b..77b2bd098a2 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleXMLHttp2Test.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleXMLHttp2Test.java
@@ -39,7 +39,7 @@ public class SolrExampleXMLHttp2Test extends SolrExampleTests {
   public SolrClient createNewSolrClient() {
 
     Http2SolrClient client =
-        new Http2SolrClient.Builder(getServerUrl())
+        new Http2SolrClient.Builder(getCoreUrl())
             .withConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)
             .withRequestWriter(new RequestWriter())
             .withResponseParser(new XMLResponseParser())
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
index e153e29598f..8bd9a193eed 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
@@ -209,7 +209,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   public void testTimeout() throws Exception {
     SolrQuery q = new SolrQuery("*:*");
     try (SolrClient client =
-        new HttpSolrClient.Builder(jetty.getBaseUrl().toString() + "/slow/foo")
+        new HttpSolrClient.Builder(getBaseUrl() + "/slow/foo")
             .withConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)
             .withSocketTimeout(2000, TimeUnit.MILLISECONDS)
             .build()) {
@@ -231,7 +231,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
         ErrorCode.UNKNOWN,
         ErrorCode.getErrorCode(status));
 
-    try (SolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/debug/foo")) {
+    try (SolrClient client = getHttpSolrClient(getBaseUrl() + "/debug/foo")) {
       DebugServlet.setErrorCode(status);
       SolrQuery q = new SolrQuery("foo");
       SolrException e = expectThrows(SolrException.class, () -> client.query(q, METHOD.GET));
@@ -244,7 +244,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   @Test
   public void testQuery() throws Exception {
     DebugServlet.clear();
-    String url = jetty.getBaseUrl().toString() + "/debug/foo";
+    String url = getBaseUrl() + "/debug/foo";
     SolrQuery q = new SolrQuery("foo");
     q.setParam("a", "\u1234");
     try (HttpSolrClient client = getHttpSolrClient(url)) {
@@ -398,7 +398,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   @Test
   public void testDelete() throws Exception {
     DebugServlet.clear();
-    String url = jetty.getBaseUrl().toString() + "/debug/foo";
+    String url = getBaseUrl() + "/debug/foo";
     try (HttpSolrClient client = getHttpSolrClient(url)) {
       expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.deleteById("id"));
 
@@ -447,7 +447,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   @Test
   public void testGetById() throws Exception {
     DebugServlet.clear();
-    try (SolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/debug/foo")) {
+    try (SolrClient client = getHttpSolrClient(getBaseUrl() + "/debug/foo")) {
       Collection<String> ids = Collections.singletonList("a");
       expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.getById("a"));
       expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.getById(ids, null));
@@ -460,7 +460,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   @Test
   public void testUpdate() throws Exception {
     DebugServlet.clear();
-    String url = jetty.getBaseUrl().toString() + "/debug/foo";
+    String url = getBaseUrl() + "/debug/foo";
 
     try (HttpSolrClient client = getHttpSolrClient(url)) {
       UpdateRequest req = new UpdateRequest();
@@ -543,7 +543,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
 
   @Test
   public void testRedirect() throws Exception {
-    final String clientUrl = jetty.getBaseUrl().toString() + "/redirect/foo";
+    final String clientUrl = getBaseUrl() + "/redirect/foo";
     SolrQuery q = new SolrQuery("*:*");
 
     // default for redirect is false.
@@ -570,7 +570,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   public void testCompression() throws Exception {
     final SolrQuery q = new SolrQuery("*:*");
 
-    final String clientUrl = jetty.getBaseUrl().toString() + "/debug/foo";
+    final String clientUrl = getBaseUrl() + "/debug/foo";
     try (SolrClient client = getHttpSolrClient(clientUrl)) {
       // verify request header gets set
       DebugServlet.clear();
@@ -596,8 +596,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
     assertNull(DebugServlet.headers.get("Accept-Encoding"));
 
     // verify server compresses output
-    HttpGet get =
-        new HttpGet(jetty.getBaseUrl().toString() + "/collection1" + "/select?q=foo&wt=xml");
+    HttpGet get = new HttpGet(getCoreUrl() + "/select?q=foo&wt=xml");
     get.setHeader("Accept-Encoding", "gzip");
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(HttpClientUtil.PROP_ALLOW_COMPRESSION, true);
@@ -622,7 +621,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
     }
 
     // verify compressed response can be handled
-    try (SolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/collection1")) {
+    try (SolrClient client = getHttpSolrClient(getCoreUrl())) {
       QueryResponse response = client.query(new SolrQuery("foo"));
       assertEquals(0, response.getStatus());
     }
@@ -631,7 +630,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   @Test
   public void testCollectionParameters() throws IOException, SolrServerException {
 
-    try (SolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString())) {
+    try (SolrClient client = getHttpSolrClient(getBaseUrl())) {
       SolrInputDocument doc = new SolrInputDocument();
       doc.addField("id", "collection");
       client.add("collection1", doc);
@@ -642,7 +641,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
           client.query("collection1", new SolrQuery("id:collection")).getResults().getNumFound());
     }
 
-    final String collection1Url = jetty.getBaseUrl().toString() + "/collection1";
+    final String collection1Url = getCoreUrl();
     try (SolrClient client = getHttpSolrClient(collection1Url)) {
       assertEquals(1, client.query(new SolrQuery("id:collection")).getResults().getNumFound());
     }
@@ -652,7 +651,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   public void testGetRawStream() throws SolrServerException, IOException {
     CloseableHttpClient client = HttpClientUtil.createClient(null);
     try (SolrClient solrClient =
-        new HttpSolrClient.Builder(jetty.getBaseUrl().toString() + "/collection1")
+        new HttpSolrClient.Builder(getCoreUrl())
             .withHttpClient(client)
             .withResponseParser(null)
             .build(); ) {
@@ -697,7 +696,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
           BasicClientCookie cookie = new BasicClientCookie(cookieName, cookieValue);
           cookie.setVersion(0);
           cookie.setPath("/");
-          cookie.setDomain(jetty.getBaseUrl().getHost());
+          cookie.setDomain(getJetty().getBaseUrl().getHost());
 
           CookieStore cookieStore = new BasicCookieStore();
           CookieSpec cookieSpec = new SolrPortAwareCookieSpecFactory().create(context);
@@ -721,7 +720,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
     HttpClientUtil.addRequestInterceptor(changeRequestInterceptor);
     HttpClientUtil.addRequestInterceptor(cookieSettingRequestInterceptor);
 
-    final String clientUrl = jetty.getBaseUrl().toString() + "/debug/foo";
+    final String clientUrl = getBaseUrl() + "/debug/foo";
     try (SolrClient server = getHttpSolrClient(clientUrl)) {
 
       SolrQuery q = new SolrQuery("foo");
@@ -786,7 +785,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   @Test
   public void testQueryString() throws Exception {
 
-    final String clientUrl = jetty.getBaseUrl().toString() + "/debug/foo";
+    final String clientUrl = getBaseUrl() + "/debug/foo";
     HttpSolrClient.Builder builder = new HttpSolrClient.Builder(clientUrl);
     try (HttpSolrClient client =
         builder.withTheseParamNamesInTheUrl(Set.of("serverOnly")).build()) {
@@ -838,7 +837,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   public void testInvariantParams() throws IOException {
     try (HttpSolrClient createdClient =
         new HttpSolrClient.Builder()
-            .withBaseSolrUrl(jetty.getBaseUrl().toString())
+            .withBaseSolrUrl(getBaseUrl())
             .withInvariantParams(SolrTestCaseJ4.params("param", "value"))
             .build()) {
       assertEquals("value", createdClient.getInvariantParams().get("param"));
@@ -846,7 +845,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
 
     try (HttpSolrClient createdClient =
         new HttpSolrClient.Builder()
-            .withBaseSolrUrl(jetty.getBaseUrl().toString())
+            .withBaseSolrUrl(getBaseUrl())
             .withInvariantParams(SolrTestCaseJ4.params("fq", "fq1", "fq", "fq2"))
             .build()) {
       assertEquals(2, createdClient.getInvariantParams().getParams("fq").length);
@@ -854,7 +853,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
 
     try (SolrClient createdClient =
         new HttpSolrClient.Builder()
-            .withBaseSolrUrl(jetty.getBaseUrl().toString())
+            .withBaseSolrUrl(getBaseUrl())
             .withKerberosDelegationToken("mydt")
             .withInvariantParams(
                 SolrTestCaseJ4.params(DelegationTokenHttpSolrClient.DELEGATION_TOKEN_PARAM, "mydt"))
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClientBadInputTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClientBadInputTest.java
index 0ded24475ab..552a48c379b 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClientBadInputTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClientBadInputTest.java
@@ -46,7 +46,7 @@ public class ConcurrentUpdateHttp2SolrClientBadInputTest extends SolrJettyTestBa
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         SolrClient client =
             new ConcurrentUpdateHttp2SolrClient.Builder(
-                    jetty.getBaseUrl().toString() + "/" + ANY_COLLECTION, http2Client)
+                    getBaseUrl() + "/" + ANY_COLLECTION, http2Client)
                 .withQueueSize(ANY_QUEUE_SIZE)
                 .withThreadCount(ANY_MAX_NUM_THREADS)
                 .build()) {
@@ -79,7 +79,7 @@ public class ConcurrentUpdateHttp2SolrClientBadInputTest extends SolrJettyTestBa
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         SolrClient client =
             new ConcurrentUpdateHttp2SolrClient.Builder(
-                    jetty.getBaseUrl().toString() + "/" + ANY_COLLECTION, http2Client)
+                    getBaseUrl() + "/" + ANY_COLLECTION, http2Client)
                 .withQueueSize(ANY_QUEUE_SIZE)
                 .withThreadCount(ANY_MAX_NUM_THREADS)
                 .build()) {
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClientTest.java
index b786ba7ed70..70804ff2654 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateHttp2SolrClientTest.java
@@ -51,7 +51,7 @@ public class ConcurrentUpdateHttp2SolrClientTest extends SolrJettyTestBase {
   public void testConcurrentUpdate() throws Exception {
     ConcurrentUpdateSolrClientTest.TestServlet.clear();
 
-    String serverUrl = jetty.getBaseUrl().toString() + "/cuss/foo";
+    String serverUrl = getBaseUrl() + "/cuss/foo";
 
     int cussThreadCount = 2;
     int cussQueueSize = 100;
@@ -126,8 +126,7 @@ public class ConcurrentUpdateHttp2SolrClientTest extends SolrJettyTestBase {
 
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         ConcurrentUpdateHttp2SolrClient concurrentClient =
-            (new ConcurrentUpdateHttp2SolrClient.Builder(
-                    jetty.getBaseUrl().toString(), http2Client))
+            (new ConcurrentUpdateHttp2SolrClient.Builder(getBaseUrl(), http2Client))
                 .withQueueSize(cussQueueSize)
                 .withThreadCount(cussThreadCount)
                 .build()) {
@@ -147,8 +146,7 @@ public class ConcurrentUpdateHttp2SolrClientTest extends SolrJettyTestBase {
 
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         ConcurrentUpdateHttp2SolrClient concurrentClient =
-            new ConcurrentUpdateHttp2SolrClient.Builder(
-                    jetty.getBaseUrl().toString() + "/collection1", http2Client)
+            new ConcurrentUpdateHttp2SolrClient.Builder(getCoreUrl(), http2Client)
                 .withQueueSize(cussQueueSize)
                 .withThreadCount(cussThreadCount)
                 .build()) {
@@ -169,7 +167,7 @@ public class ConcurrentUpdateHttp2SolrClientTest extends SolrJettyTestBase {
 
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         ConcurrentUpdateHttp2SolrClient concurrentClient =
-            new ConcurrentUpdateHttp2SolrClient.Builder(jetty.getBaseUrl().toString(), http2Client)
+            new ConcurrentUpdateHttp2SolrClient.Builder(getBaseUrl(), http2Client)
                 .withQueueSize(cussQueueSize)
                 .withThreadCount(cussThreadCount)
                 .setPollQueueTime(0, TimeUnit.MILLISECONDS)
@@ -207,8 +205,7 @@ public class ConcurrentUpdateHttp2SolrClientTest extends SolrJettyTestBase {
 
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         ConcurrentUpdateHttp2SolrClient concurrentClient =
-            new ConcurrentUpdateHttp2SolrClient.Builder(
-                    jetty.getBaseUrl().toString() + "/collection1", http2Client)
+            new ConcurrentUpdateHttp2SolrClient.Builder(getCoreUrl(), http2Client)
                 .withQueueSize(cussQueueSize)
                 .withThreadCount(cussThreadCount)
                 .build()) {
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBadInputTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBadInputTest.java
index 371ea543e99..17e74924a47 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBadInputTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBadInputTest.java
@@ -43,7 +43,7 @@ public class ConcurrentUpdateSolrClientBadInputTest extends SolrJettyTestBase {
   @Test
   public void testDeleteByIdReportsInvalidIdLists() throws Exception {
     try (SolrClient client =
-        new ConcurrentUpdateSolrClient.Builder(jetty.getBaseUrl().toString() + "/" + ANY_COLLECTION)
+        new ConcurrentUpdateSolrClient.Builder(getBaseUrl() + "/" + ANY_COLLECTION)
             .withQueueSize(ANY_QUEUE_SIZE)
             .withThreadCount(ANY_MAX_NUM_THREADS)
             .build()) {
@@ -74,7 +74,7 @@ public class ConcurrentUpdateSolrClientBadInputTest extends SolrJettyTestBase {
     }
 
     try (SolrClient client =
-        new ConcurrentUpdateSolrClient.Builder(jetty.getBaseUrl().toString())
+        new ConcurrentUpdateSolrClient.Builder(getBaseUrl())
             .withQueueSize(ANY_QUEUE_SIZE)
             .withThreadCount(ANY_MAX_NUM_THREADS)
             .build()) {
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java
index ab7a3a7ef86..6cff2257f8d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java
@@ -140,7 +140,7 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
   public void testConcurrentUpdate() throws Exception {
     TestServlet.clear();
 
-    String serverUrl = jetty.getBaseUrl().toString() + "/cuss/foo";
+    String serverUrl = getBaseUrl() + "/cuss/foo";
 
     int cussThreadCount = 2;
     int cussQueueSize = 100;
@@ -208,7 +208,7 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
     int cussQueueSize = 10;
 
     try (ConcurrentUpdateSolrClient concurrentClient =
-        (new ConcurrentUpdateSolrClient.Builder(jetty.getBaseUrl().toString()))
+        (new ConcurrentUpdateSolrClient.Builder(getBaseUrl()))
             .withQueueSize(cussQueueSize)
             .withThreadCount(cussThreadCount)
             .build()) {
@@ -227,7 +227,7 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
     }
 
     try (ConcurrentUpdateSolrClient concurrentClient =
-        (new ConcurrentUpdateSolrClient.Builder(jetty.getBaseUrl().toString() + "/collection1"))
+        (new ConcurrentUpdateSolrClient.Builder(getCoreUrl()))
             .withQueueSize(cussQueueSize)
             .withThreadCount(cussThreadCount)
             .build()) {
@@ -247,7 +247,7 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
     int expected = numDocs * numRunnables;
 
     try (ConcurrentUpdateSolrClient concurrentClient =
-        (new ConcurrentUpdateSolrClient.Builder(jetty.getBaseUrl().toString()))
+        (new ConcurrentUpdateSolrClient.Builder(getBaseUrl()))
             .withQueueSize(cussQueueSize)
             .withThreadCount(cussThreadCount)
             .withPollQueueTime(0)
@@ -283,7 +283,7 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
     }
 
     try (ConcurrentUpdateSolrClient concurrentClient =
-        (new ConcurrentUpdateSolrClient.Builder(jetty.getBaseUrl().toString() + "/collection1"))
+        (new ConcurrentUpdateSolrClient.Builder(getCoreUrl()))
             .withQueueSize(cussQueueSize)
             .withThreadCount(cussThreadCount)
             .build()) {
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 78ce61a3099..d881463cd10 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
@@ -53,16 +53,14 @@ public class Http2SolrClientCompatibilityTest extends SolrJettyTestBase {
     createAndStartJetty(legacyExampleCollection1SolrHome(), jettyConfig);
 
     try (Http2SolrClient client =
-        new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo")
-            .useHttp1_1(true)
-            .build()) {
+        new Http2SolrClient.Builder(getBaseUrl() + "/debug/foo").useHttp1_1(true).build()) {
       assertTrue(client.getHttpClient().getTransport() instanceof HttpClientTransportOverHTTP);
       try {
         client.query(new SolrQuery("*:*"), SolrRequest.METHOD.GET);
       } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
       }
     } finally {
-      afterSolrJettyTestBase();
+      solrClientTestRule.reset();
     }
   }
 
@@ -76,16 +74,14 @@ public class Http2SolrClientCompatibilityTest extends SolrJettyTestBase {
     createAndStartJetty(legacyExampleCollection1SolrHome(), jettyConfig);
 
     try (Http2SolrClient client =
-        new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo")
-            .useHttp1_1(true)
-            .build()) {
+        new Http2SolrClient.Builder(getBaseUrl() + "/debug/foo").useHttp1_1(true).build()) {
       assertTrue(client.getHttpClient().getTransport() instanceof HttpClientTransportOverHTTP);
       try {
         client.query(new SolrQuery("*:*"), SolrRequest.METHOD.GET);
       } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
       }
     } finally {
-      afterSolrJettyTestBase();
+      solrClientTestRule.reset();
     }
   }
 
@@ -102,7 +98,7 @@ public class Http2SolrClientCompatibilityTest extends SolrJettyTestBase {
 
     System.clearProperty("solr.http1");
     try (Http2SolrClient client =
-        new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo").build()) {
+        new Http2SolrClient.Builder(getBaseUrl() + "/debug/foo").build()) {
       assertTrue(client.getHttpClient().getTransport() instanceof HttpClientTransportOverHTTP2);
       try {
         client.query(new SolrQuery("*:*"), SolrRequest.METHOD.GET);
@@ -113,7 +109,7 @@ public class Http2SolrClientCompatibilityTest extends SolrJettyTestBase {
         // expected
       }
     } finally {
-      afterSolrJettyTestBase();
+      solrClientTestRule.reset();
     }
   }
 }
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 09f3601c47b..9b398160193 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
@@ -17,17 +17,20 @@
 package org.apache.solr.client.solrj.impl;
 
 import com.carrotsearch.randomizedtesting.RandomizedTest;
-import java.nio.file.Path;
 import java.util.Properties;
-import org.apache.solr.SolrJettyTestBase;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.embedded.JettyConfig;
-import org.apache.solr.embedded.JettySolrRunner;
+import org.apache.solr.util.ExternalPaths;
+import org.apache.solr.util.SolrJettyTestRule;
 import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Test;
 
-public class Http2SolrClientProxyTest extends SolrJettyTestBase {
+public class Http2SolrClientProxyTest extends SolrTestCaseJ4 {
+
+  @ClassRule public static SolrJettyTestRule solrClientTestRule = new SolrJettyTestRule();
 
   // TODO add SSL test
 
@@ -35,40 +38,22 @@ public class Http2SolrClientProxyTest extends SolrJettyTestBase {
   public static void beforeTest() throws Exception {
     RandomizedTest.assumeFalse(sslConfig.isSSLMode());
 
-    JettyConfig jettyConfig =
-        JettyConfig.builder().withSSLConfig(sslConfig.buildServerSSLConfig()).build();
-    createAndStartJettyWithProxy(legacyExampleCollection1SolrHome(), new Properties(), jettyConfig);
-  }
-
-  public static JettySolrRunner createAndStartJettyWithProxy(
-      String solrHome, Properties nodeProperties, JettyConfig jettyConfig) throws Exception {
-
-    initCore(null, null, solrHome);
-
-    Path coresDir = createTempDir().resolve("cores");
-
-    Properties props = new Properties();
-    props.setProperty("name", DEFAULT_TEST_CORENAME);
-    props.setProperty("configSet", "collection1");
-    props.setProperty("config", "${solrconfig:solrconfig.xml}");
-    props.setProperty("schema", "${schema:schema.xml}");
-
-    writeCoreProperties(coresDir.resolve("core"), props, "RestTestBase");
-
-    Properties nodeProps = new Properties(nodeProperties);
-    nodeProps.setProperty("coreRootDirectory", coresDir.toString());
-    nodeProps.setProperty("configSetBaseDir", solrHome);
-
-    jetty = new JettySolrRunner(solrHome, nodeProps, jettyConfig, true);
-    jetty.start();
-    port = jetty.getLocalPort();
-    return jetty;
+    solrClientTestRule.enableProxy();
+    solrClientTestRule.startSolr(
+        createTempDir(),
+        new Properties(),
+        JettyConfig.builder().withSSLConfig(sslConfig.buildServerSSLConfig()).build());
+    // Actually only need extremely minimal configSet but just use the default
+    solrClientTestRule
+        .newCollection()
+        .withConfigSet(ExternalPaths.DEFAULT_CONFIGSET) // TODO should be default for empty home
+        .create();
   }
 
   /** Setup a simple http proxy and verify a request works */
   @Test
   public void testProxy() throws Exception {
-    var proxy = jetty.getProxy();
+    var proxy = solrClientTestRule.getJetty().getProxy();
     assertNotNull(proxy);
 
     String host = proxy.getUrl().getHost();
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 c8aaf736544..263f9aa93e1 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
@@ -36,6 +36,7 @@ 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.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -203,8 +204,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   public void testTimeout() throws Exception {
     SolrQuery q = new SolrQuery("*:*");
     try (Http2SolrClient client =
-        getHttp2SolrClientBuilder(
-                jetty.getBaseUrl().toString() + "/slow/foo", DEFAULT_CONNECTION_TIMEOUT, 2000)
+        getHttp2SolrClientBuilder(getBaseUrl() + "/slow/foo", DEFAULT_CONNECTION_TIMEOUT, 2000)
             .build()) {
       client.query(q, SolrRequest.METHOD.GET);
       fail("No exception thrown.");
@@ -217,8 +217,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   public void test0IdleTimeout() throws Exception {
     SolrQuery q = new SolrQuery("*:*");
     try (Http2SolrClient client =
-        getHttp2SolrClientBuilder(
-                jetty.getBaseUrl().toString() + "/debug/foo", DEFAULT_CONNECTION_TIMEOUT, 0)
+        getHttp2SolrClientBuilder(getBaseUrl() + "/debug/foo", DEFAULT_CONNECTION_TIMEOUT, 0)
             .build()) {
       try {
         client.query(q, SolrRequest.METHOD.GET);
@@ -231,8 +230,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   public void testRequestTimeout() throws Exception {
     SolrQuery q = new SolrQuery("*:*");
     try (Http2SolrClient client =
-        getHttp2SolrClientBuilder(
-                jetty.getBaseUrl().toString() + "/slow/foo", DEFAULT_CONNECTION_TIMEOUT, 0)
+        getHttp2SolrClientBuilder(getBaseUrl() + "/slow/foo", DEFAULT_CONNECTION_TIMEOUT, 0)
             .withRequestTimeout(500, TimeUnit.MILLISECONDS)
             .build()) {
       client.query(q, SolrRequest.METHOD.GET);
@@ -256,7 +254,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
         SolrException.ErrorCode.getErrorCode(status));
 
     try (Http2SolrClient client =
-        new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo").build()) {
+        new Http2SolrClient.Builder(getBaseUrl() + "/debug/foo").build()) {
       DebugServlet.setErrorCode(status);
       try {
         SolrQuery q = new SolrQuery("foo");
@@ -288,13 +286,13 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       try {
         // if client base url is null, request url will be used in exception message
         SolrPing ping = new SolrPing();
-        ping.setBasePath(jetty.getBaseUrl().toString() + "/debug/foo");
+        ping.setBasePath(getBaseUrl() + "/debug/foo");
         client.request(ping);
 
         fail("Didn't get excepted exception from oversided request");
       } catch (SolrException e) {
         assertEquals("Unexpected exception status code", status, e.code());
-        assertTrue(e.getMessage().contains(jetty.getBaseUrl().toString()));
+        assertTrue(e.getMessage().contains(getBaseUrl()));
       }
     } finally {
       DebugServlet.clear();
@@ -304,7 +302,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   @Test
   public void testQuery() throws Exception {
     DebugServlet.clear();
-    String url = jetty.getBaseUrl().toString() + "/debug/foo";
+    String url = getBaseUrl() + "/debug/foo";
     SolrQuery q = new SolrQuery("foo");
     q.setParam("a", "\u1234");
     try (Http2SolrClient client = new Http2SolrClient.Builder(url).build()) {
@@ -434,7 +432,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   @Test
   public void testDelete() throws Exception {
     DebugServlet.clear();
-    String url = jetty.getBaseUrl().toString() + "/debug/foo";
+    String url = getBaseUrl() + "/debug/foo";
     try (Http2SolrClient client = new Http2SolrClient.Builder(url).build()) {
       try {
         client.deleteById("id");
@@ -479,7 +477,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   public void testGetById() throws Exception {
     DebugServlet.clear();
     try (Http2SolrClient client =
-        new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo").build()) {
+        new Http2SolrClient.Builder(getBaseUrl() + "/debug/foo").build()) {
       Collection<String> ids = Collections.singletonList("a");
       try {
         client.getById("a");
@@ -506,7 +504,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   @Test
   public void testUpdate() throws Exception {
     DebugServlet.clear();
-    String url = jetty.getBaseUrl().toString() + "/debug/foo";
+    String url = getBaseUrl() + "/debug/foo";
     UpdateRequest req = new UpdateRequest();
     req.add(new SolrInputDocument());
     req.setParam("a", "\u1234");
@@ -586,7 +584,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
 
   @Test
   public void testFollowRedirect() throws Exception {
-    final String clientUrl = jetty.getBaseUrl().toString() + "/redirect/foo";
+    final String clientUrl = getBaseUrl() + "/redirect/foo";
     try (Http2SolrClient client =
         new Http2SolrClient.Builder(clientUrl).withFollowRedirects(true).build()) {
       SolrQuery q = new SolrQuery("*:*");
@@ -596,7 +594,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
 
   @Test
   public void testDoNotFollowRedirect() throws Exception {
-    final String clientUrl = jetty.getBaseUrl().toString() + "/redirect/foo";
+    final String clientUrl = getBaseUrl() + "/redirect/foo";
     try (Http2SolrClient client =
         new Http2SolrClient.Builder(clientUrl).withFollowRedirects(false).build()) {
       SolrQuery q = new SolrQuery("*:*");
@@ -608,7 +606,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
 
   @Test
   public void testRedirectSwapping() throws Exception {
-    final String clientUrl = jetty.getBaseUrl().toString() + "/redirect/foo";
+    final String clientUrl = getBaseUrl() + "/redirect/foo";
     SolrQuery q = new SolrQuery("*:*");
 
     // default for follow redirects is false
@@ -636,8 +634,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   @Test
   public void testCollectionParameters() throws IOException, SolrServerException {
 
-    try (Http2SolrClient client =
-        new Http2SolrClient.Builder(jetty.getBaseUrl().toString()).build()) {
+    try (Http2SolrClient client = new Http2SolrClient.Builder(getBaseUrl()).build()) {
       SolrInputDocument doc = new SolrInputDocument();
       doc.addField("id", "collection");
       client.add("collection1", doc);
@@ -648,7 +645,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
           client.query("collection1", new SolrQuery("id:collection")).getResults().getNumFound());
     }
 
-    final String collection1Url = jetty.getBaseUrl().toString() + "/collection1";
+    final String collection1Url = getCoreUrl();
     try (Http2SolrClient client = new Http2SolrClient.Builder(collection1Url).build()) {
       assertEquals(1, client.query(new SolrQuery("id:collection")).getResults().getNumFound());
     }
@@ -687,7 +684,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   @Test
   public void testQueryString() throws Exception {
 
-    final String clientUrl = jetty.getBaseUrl().toString() + "/debug/foo";
+    final String clientUrl = getBaseUrl() + "/debug/foo";
     UpdateRequest req = new UpdateRequest();
 
     try (Http2SolrClient client =
@@ -820,7 +817,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   @Test
   public void testSetCredentialsExplicitly() {
     try (Http2SolrClient client =
-        new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo")
+        new Http2SolrClient.Builder(getBaseUrl() + "/debug/foo")
             .withBasicAuthCredentials("foo", "explicit")
             .build(); ) {
       QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
@@ -854,7 +851,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
     PreemptiveBasicAuthClientBuilderFactory.setDefaultSolrParams(
         new PreemptiveBasicAuthClientBuilderFactory.CredentialsResolver().defaultParams);
     try (Http2SolrClient client =
-        new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo").build(); ) {
+        new Http2SolrClient.Builder(getBaseUrl() + "/debug/foo").build(); ) {
       QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
       DebugServlet.addResponseHeader(
           WWWAuthenticationProtocolHandler.NAME, "Basic realm=\"Debug Servlet\"");
@@ -883,7 +880,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   @Test
   public void testPerRequestCredentialsWin() {
     try (Http2SolrClient client =
-        new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo")
+        new Http2SolrClient.Builder(getBaseUrl() + "/debug/foo")
             .withBasicAuthCredentials("foo2", "explicit")
             .build(); ) {
       QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
@@ -913,7 +910,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   @Test
   public void testNoCredentials() {
     try (Http2SolrClient client =
-        new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo").build(); ) {
+        new Http2SolrClient.Builder(getBaseUrl() + "/debug/foo").build(); ) {
       QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
       try {
         ignoreException("Error from server");
@@ -932,7 +929,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
   public void testBadHttpFactory() {
     System.setProperty(HttpClientUtil.SYS_PROP_HTTP_CLIENT_BUILDER_FACTORY, "FakeClassName");
     try {
-      client = new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo").build();
+      SolrClient client = new Http2SolrClient.Builder(getBaseUrl() + "/debug/foo").build();
       fail("Expecting exception");
     } catch (RuntimeException e) {
       assertTrue(e.getMessage().contains("Unable to instantiate"));
@@ -944,9 +941,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
     DebugServlet.clear();
     try (Http2SolrClient client =
         getHttp2SolrClientBuilder(
-                jetty.getBaseUrl().toString() + "/debug/foo",
-                DEFAULT_CONNECTION_TIMEOUT,
-                DEFAULT_CONNECTION_TIMEOUT)
+                getBaseUrl() + "/debug/foo", DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)
             .build()) {
       GenericSolrRequest req =
           new GenericSolrRequest(SolrRequest.METHOD.GET, "/select", params("q", "*:*"));
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientBadInputTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientBadInputTest.java
index fc1fd5c05dc..e3e6d513beb 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientBadInputTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientBadInputTest.java
@@ -41,8 +41,7 @@ public class HttpSolrClientBadInputTest extends SolrJettyTestBase {
 
   @Test
   public void testDeleteByIdReportsInvalidIdLists() throws Exception {
-    try (SolrClient client =
-        getHttpSolrClient(jetty.getBaseUrl().toString() + "/" + ANY_COLLECTION)) {
+    try (SolrClient client = getHttpSolrClient(getBaseUrl() + "/" + ANY_COLLECTION)) {
       assertExceptionThrownWithMessageContaining(
           IllegalArgumentException.class,
           List.of("ids", "null"),
@@ -69,7 +68,7 @@ public class HttpSolrClientBadInputTest extends SolrJettyTestBase {
           });
     }
 
-    try (SolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString())) {
+    try (SolrClient client = getHttpSolrClient(getBaseUrl())) {
       assertExceptionThrownWithMessageContaining(
           IllegalArgumentException.class,
           List.of("ids", "null"),
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java
index 0c0812a1cf3..57b03eff90a 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java
@@ -17,6 +17,7 @@
 package org.apache.solr.client.solrj.impl;
 
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -34,41 +35,28 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
-import org.apache.solr.embedded.JettySolrRunner;
-import org.junit.AfterClass;
+import org.apache.solr.util.SolrJettyTestRule;
 import org.junit.BeforeClass;
+import org.junit.ClassRule;
 
 public class HttpSolrClientConPoolTest extends SolrJettyTestBase {
 
-  protected static JettySolrRunner yetty;
-  private static String fooUrl;
-  private static String barUrl;
+  @ClassRule public static SolrJettyTestRule secondJetty = new SolrJettyTestRule();
+  private static String fooUrl; // first Jetty URL
+  private static String barUrl; // second Jetty URL
 
   @BeforeClass
   public static void beforeTest() throws Exception {
     createAndStartJetty(legacyExampleCollection1SolrHome());
-    // stealing the first made jetty
-    yetty = jetty;
-    barUrl = yetty.getBaseUrl().toString() + "/" + "collection1";
+    fooUrl = getBaseUrl() + "/" + "collection1";
 
-    createAndStartJetty(legacyExampleCollection1SolrHome());
-
-    fooUrl = jetty.getBaseUrl().toString() + "/" + "collection1";
-  }
-
-  @AfterClass
-  public static void stopYetty() throws Exception {
-    if (null != yetty) {
-      yetty.stop();
-      yetty = null;
-    }
+    secondJetty.startSolr(Path.of(legacyExampleCollection1SolrHome()));
+    barUrl = secondJetty.getBaseUrl() + "/" + "collection1";
   }
 
   public void testPoolSize() throws SolrServerException, IOException {
     PoolingHttpClientConnectionManager pool = HttpClientUtil.createPoolingConnectionManager();
 
-    final String fooUrl = jetty.getBaseUrl().toString() + "/" + "collection1";
-    final String barUrl = yetty.getBaseUrl().toString() + "/" + "collection1";
     CloseableHttpClient httpClient =
         HttpClientUtil.createClient(
             new ModifiableSolrParams(), pool, false /* let client shutdown it*/);
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientSSLAuthConPoolTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientSSLAuthConPoolTest.java
index fda526b5298..ba78c0b4c7e 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientSSLAuthConPoolTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientSSLAuthConPoolTest.java
@@ -26,7 +26,7 @@ public class HttpSolrClientSSLAuthConPoolTest extends HttpSolrClientConPoolTest
 
   @BeforeClass
   public static void checkUrls() throws Exception {
-    URL[] urls = new URL[] {jetty.getBaseUrl(), yetty.getBaseUrl()};
+    URL[] urls = new URL[] {getJetty().getBaseUrl(), secondJetty.getJetty().getBaseUrl()};
     for (URL u : urls) {
       assertEquals("expect https urls ", "https", u.getProtocol());
     }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/LBHttpSolrClientBadInputTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/LBHttpSolrClientBadInputTest.java
index d030b3e98b7..0432c26add5 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/LBHttpSolrClientBadInputTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/LBHttpSolrClientBadInputTest.java
@@ -42,7 +42,7 @@ public class LBHttpSolrClientBadInputTest extends SolrJettyTestBase {
   public void testDeleteByIdReportsInvalidIdLists() throws Exception {
     try (SolrClient client =
         new LBHttpSolrClient.Builder()
-            .withBaseSolrUrls(jetty.getBaseUrl().toString() + "/" + ANY_COLLECTION)
+            .withBaseSolrUrls(getBaseUrl() + "/" + ANY_COLLECTION)
             .build()) {
       assertExceptionThrownWithMessageContaining(
           IllegalArgumentException.class,
@@ -71,7 +71,7 @@ public class LBHttpSolrClientBadInputTest extends SolrJettyTestBase {
     }
 
     try (SolrClient client =
-        new LBHttpSolrClient.Builder().withBaseSolrUrls(jetty.getBaseUrl().toString()).build()) {
+        new LBHttpSolrClient.Builder().withBaseSolrUrls(getBaseUrl()).build()) {
       assertExceptionThrownWithMessageContaining(
           IllegalArgumentException.class,
           List.of("ids", "null"),
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java
index 0b87c7d6ef5..2be715d469d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java
@@ -131,10 +131,7 @@ public class SchemaTest extends RestTestBase {
 
   @After
   public void cleanup() throws Exception {
-    if (jetty != null) {
-      jetty.stop();
-      jetty = null;
-    }
+    solrClientTestRule.reset();
     if (restTestHarness != null) {
       restTestHarness.close();
     }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/response/NoOpResponseParserTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/response/NoOpResponseParserTest.java
index 4cb7789e4ff..a97c36676f8 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/response/NoOpResponseParserTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/response/NoOpResponseParserTest.java
@@ -71,7 +71,7 @@ public class NoOpResponseParserTest extends SolrJettyTestBase {
   public void testQueryParse() throws Exception {
 
     try (SolrClient client =
-        new HttpSolrClient.Builder(getServerUrl())
+        new HttpSolrClient.Builder(getCoreUrl())
             .withResponseParser(new NoOpResponseParser())
             .build()) {
       SolrQuery query = new SolrQuery("id:1234");
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestSuggesterResponse.java b/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestSuggesterResponse.java
index 7be940eed2f..3f907622a58 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestSuggesterResponse.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestSuggesterResponse.java
@@ -119,8 +119,8 @@ public class TestSuggesterResponse extends SolrJettyTestBase {
   }
 
   private void addSampleDocs() throws SolrServerException, IOException {
-    client.deleteByQuery("*:*");
-    client.commit(true, true);
+    getSolrClient().deleteByQuery("*:*");
+    getSolrClient().commit(true, true);
     SolrInputDocument doc = new SolrInputDocument();
     doc.setField("id", "111");
     doc.setField(field, "Computer");
@@ -130,10 +130,10 @@ public class TestSuggesterResponse extends SolrJettyTestBase {
     SolrInputDocument doc3 = new SolrInputDocument();
     doc3.setField("id", "333");
     doc3.setField(field, "Laptop");
-    client.add(doc);
-    client.add(doc2);
-    client.add(doc3);
-    client.commit(true, true);
+    getSolrClient().add(doc);
+    getSolrClient().add(doc2);
+    getSolrClient().add(doc3);
+    getSolrClient().commit(true, true);
   }
 
   /*
@@ -143,7 +143,7 @@ public class TestSuggesterResponse extends SolrJettyTestBase {
     final ResponseParser randomParser =
         random().nextBoolean() ? new BinaryResponseParser() : new XMLResponseParser();
     return new HttpSolrClient.Builder()
-        .withBaseSolrUrl(jetty.getBaseUrl().toString() + "/collection1")
+        .withBaseSolrUrl(getCoreUrl())
         .withResponseParser(randomParser)
         .build();
   }
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java b/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
index 2042b24a498..a4b08656fd7 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
@@ -17,44 +17,24 @@
 package org.apache.solr;
 
 import java.io.File;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.UncheckedIOException;
-import java.io.Writer;
-import java.lang.invoke.MethodHandles;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.Properties;
 import java.util.SortedMap;
 import org.apache.commons.io.file.PathUtils;
 import org.apache.http.client.HttpClient;
-import org.apache.lucene.tests.util.LuceneTestCase;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.embedded.JettyConfig;
 import org.apache.solr.embedded.JettySolrRunner;
-import org.apache.solr.util.DirectoryUtil;
-import org.apache.solr.util.ExternalPaths;
+import org.apache.solr.util.SolrJettyTestRule;
 import org.eclipse.jetty.servlet.ServletHolder;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.junit.ClassRule;
 
+@Deprecated // just use SolrJettyTestRule
 public abstract class SolrJettyTestBase extends SolrTestCaseJ4 {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  @ClassRule public static SolrJettyTestRule solrClientTestRule = new SolrJettyTestRule();
 
-  @BeforeClass
-  public static void beforeSolrJettyTestBase() throws Exception {}
-
-  public static JettySolrRunner jetty;
-  public static int port;
-  public static SolrClient client = null;
-  public static String context;
-
-  public static JettySolrRunner createAndStartJetty(
+  protected static JettySolrRunner createAndStartJetty(
       String solrHome,
       String configFile,
       String schemaFile,
@@ -64,8 +44,7 @@ public abstract class SolrJettyTestBase extends SolrTestCaseJ4 {
       throws Exception {
     // creates the data dir
 
-    context = context == null ? "/solr" : context;
-    SolrJettyTestBase.context = context;
+    assert context == null || context.equals("/solr"); // deprecated
 
     JettyConfig jettyConfig =
         JettyConfig.builder()
@@ -85,28 +64,27 @@ public abstract class SolrJettyTestBase extends SolrTestCaseJ4 {
     return createAndStartJetty(solrHome, nodeProps, jettyConfig);
   }
 
-  public static JettySolrRunner createAndStartJetty(
+  protected static JettySolrRunner createAndStartJetty(
       String solrHome, String configFile, String context) throws Exception {
     return createAndStartJetty(solrHome, configFile, null, context, true, null);
   }
 
-  public static JettySolrRunner createAndStartJetty(String solrHome, JettyConfig jettyConfig)
+  protected static JettySolrRunner createAndStartJetty(String solrHome, JettyConfig jettyConfig)
       throws Exception {
+
     return createAndStartJetty(solrHome, new Properties(), jettyConfig);
   }
 
-  public static JettySolrRunner createAndStartJetty(String solrHome) throws Exception {
+  protected static JettySolrRunner createAndStartJetty(String solrHome) throws Exception {
     return createAndStartJetty(
         solrHome,
         new Properties(),
         JettyConfig.builder().withSSLConfig(sslConfig.buildServerSSLConfig()).build());
   }
 
-  public static JettySolrRunner createAndStartJetty(
+  protected static JettySolrRunner createAndStartJetty(
       String solrHome, Properties nodeProperties, JettyConfig jettyConfig) throws Exception {
 
-    initCore(null, null, solrHome);
-
     Path coresDir = createTempDir().resolve("cores");
 
     Properties props = new Properties();
@@ -121,36 +99,26 @@ public abstract class SolrJettyTestBase extends SolrTestCaseJ4 {
     nodeProps.setProperty("coreRootDirectory", coresDir.toString());
     nodeProps.setProperty("configSetBaseDir", solrHome);
 
-    jetty = new JettySolrRunner(solrHome, nodeProps, jettyConfig);
-    jetty.start();
-    port = jetty.getLocalPort();
-    log.info("Jetty Assigned Port#{}", port);
-    return jetty;
+    solrClientTestRule.startSolr(Path.of(solrHome), nodeProps, jettyConfig);
+    return getJetty();
   }
 
-  protected String getServerUrl() {
-    return jetty.getBaseUrl().toString() + "/" + DEFAULT_TEST_CORENAME;
+  protected static JettySolrRunner getJetty() {
+    return solrClientTestRule.getJetty();
   }
 
-  @After
-  public synchronized void afterClass() throws Exception {
-    if (client != null) client.close();
-    client = null;
+  /** URL to Solr */
+  protected static String getBaseUrl() {
+    return solrClientTestRule.getBaseUrl();
   }
 
-  @AfterClass
-  public static void afterSolrJettyTestBase() throws Exception {
-    if (jetty != null) {
-      jetty.stop();
-      jetty = null;
-    }
+  /** URL to the core */
+  protected static String getCoreUrl() {
+    return getBaseUrl() + "/" + DEFAULT_TEST_CORENAME;
   }
 
-  public synchronized SolrClient getSolrClient() {
-    if (client == null) {
-      client = createNewSolrClient();
-    }
-    return client;
+  protected SolrClient getSolrClient() {
+    return solrClientTestRule.getSolrClient();
   }
 
   /**
@@ -159,10 +127,10 @@ public abstract class SolrJettyTestBase extends SolrTestCaseJ4 {
    * options.
    */
   public SolrClient createNewSolrClient() {
-    return new HttpSolrClient.Builder(getServerUrl()).build();
+    return new HttpSolrClient.Builder(getCoreUrl()).build();
   }
 
-  public HttpClient getHttpClient() {
+  protected HttpClient getHttpClient() {
     HttpSolrClient client = (HttpSolrClient) getSolrClient();
     return client.getHttpClient();
   }
@@ -170,46 +138,17 @@ public abstract class SolrJettyTestBase extends SolrTestCaseJ4 {
   // Sets up the necessary config files for Jetty. At least some tests require that the solrconfig
   // from the test file directory are used, but some also require that the solr.xml file be
   // explicitly there as of SOLR-4817
-  public static void setupJettyTestHome(File solrHome, String collection) throws Exception {
+  @Deprecated // Instead use a basic config + whatever is needed or default config
+  protected static void setupJettyTestHome(File solrHome, String collection) throws Exception {
+    // TODO remove these sys props!
+    System.setProperty("solr.test.sys.prop1", "propone");
+    System.setProperty("solr.test.sys.prop2", "proptwo");
     copySolrHomeToTemp(solrHome, collection);
   }
 
-  public static void cleanUpJettyHome(File solrHome) throws Exception {
+  protected static void cleanUpJettyHome(File solrHome) throws Exception {
     if (solrHome.exists()) {
       PathUtils.deleteDirectory(solrHome.toPath());
     }
   }
-
-  public static String legacyExampleCollection1SolrHome() {
-    String sourceHome = ExternalPaths.SOURCE_HOME;
-    if (sourceHome == null)
-      throw new IllegalStateException(
-          "No source home! Cannot create the legacy example solr home directory.");
-
-    try {
-      Path tempSolrHome = LuceneTestCase.createTempDir();
-      Path serverSolr = tempSolrHome.getFileSystem().getPath(sourceHome, "server", "solr");
-      Files.copy(serverSolr.resolve("solr.xml"), tempSolrHome.resolve("solr.xml"));
-
-      Path sourceConfig = serverSolr.resolve("configsets").resolve("sample_techproducts_configs");
-      Path collection1Dir = tempSolrHome.resolve("collection1");
-
-      DirectoryUtil.copyDirectoryContents(
-          sourceConfig.resolve("conf"), collection1Dir.resolve("conf"));
-
-      Properties props = new Properties();
-      props.setProperty("name", "collection1");
-      try (Writer writer =
-          new OutputStreamWriter(
-              Files.newOutputStream(collection1Dir.resolve("core.properties")),
-              StandardCharsets.UTF_8)) {
-        props.store(writer, null);
-      }
-      return tempSolrHome.toString();
-    } catch (RuntimeException e) {
-      throw e;
-    } catch (IOException e) {
-      throw new UncheckedIOException(e);
-    }
-  }
 }
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index fc4a6ac27fb..32a629824cc 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -32,6 +32,7 @@ import java.io.OutputStreamWriter;
 import java.io.Reader;
 import java.io.StringReader;
 import java.io.StringWriter;
+import java.io.UncheckedIOException;
 import java.io.Writer;
 import java.lang.annotation.Documented;
 import java.lang.annotation.ElementType;
@@ -79,6 +80,7 @@ import org.apache.logging.log4j.Level;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.tests.analysis.MockAnalyzer;
 import org.apache.lucene.tests.analysis.MockTokenizer;
+import org.apache.lucene.tests.util.LuceneTestCase;
 import org.apache.lucene.tests.util.LuceneTestCase.SuppressFileSystems;
 import org.apache.lucene.tests.util.TestUtil;
 import org.apache.lucene.util.Constants;
@@ -136,7 +138,9 @@ import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
 import org.apache.solr.update.processor.DistributedZkUpdateProcessor;
 import org.apache.solr.update.processor.UpdateRequestProcessor;
 import org.apache.solr.util.BaseTestHarness;
+import org.apache.solr.util.DirectoryUtil;
 import org.apache.solr.util.ErrorLogMuter;
+import org.apache.solr.util.ExternalPaths;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.RandomizeSSL;
 import org.apache.solr.util.RandomizeSSL.SSLRandomizer;
@@ -2285,6 +2289,8 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
   // Creates a consistent configuration, _including_ solr.xml at dstRoot. Creates collection1/conf
   // and copies the stock files in there.
 
+  /** Copies the test collection1 config into {@code dstRoot}/{@code collection}/conf */
+  @Deprecated // Instead use a basic config + whatever is needed or default config
   public static void copySolrHomeToTemp(File dstRoot, String collection) throws IOException {
     Path subHome = dstRoot.toPath().resolve(collection).resolve("conf");
     Files.createDirectories(subHome);
@@ -2312,6 +2318,41 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
     Files.copy(top.resolve("synonyms.txt"), subHome.resolve("synonyms.txt"));
   }
 
+  /** Creates a temp solr home using sample_techproducts_configs. Returns the home path. */
+  @Deprecated // Instead use a basic config + whatever is needed or default config
+  public static String legacyExampleCollection1SolrHome() {
+    String sourceHome = ExternalPaths.SOURCE_HOME;
+    if (sourceHome == null)
+      throw new IllegalStateException(
+          "No source home! Cannot create the legacy example solr home directory.");
+
+    try {
+      Path tempSolrHome = LuceneTestCase.createTempDir();
+      Path serverSolr = tempSolrHome.getFileSystem().getPath(sourceHome, "server", "solr");
+      Files.copy(serverSolr.resolve("solr.xml"), tempSolrHome.resolve("solr.xml"));
+
+      Path sourceConfig = serverSolr.resolve("configsets").resolve("sample_techproducts_configs");
+      Path collection1Dir = tempSolrHome.resolve("collection1");
+
+      DirectoryUtil.copyDirectoryContents(
+          sourceConfig.resolve("conf"), collection1Dir.resolve("conf"));
+
+      Properties props = new Properties();
+      props.setProperty("name", "collection1");
+      try (Writer writer =
+          new OutputStreamWriter(
+              Files.newOutputStream(collection1Dir.resolve("core.properties")),
+              StandardCharsets.UTF_8)) {
+        props.store(writer, null);
+      }
+      return tempSolrHome.toString();
+    } catch (RuntimeException e) {
+      throw e;
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
   public boolean compareSolrDocument(Object expected, Object actual) {
 
     if (!(expected instanceof SolrDocument) || !(actual instanceof SolrDocument)) {
diff --git a/solr/test-framework/src/java/org/apache/solr/util/EmbeddedSolrServerTestRule.java b/solr/test-framework/src/java/org/apache/solr/util/EmbeddedSolrServerTestRule.java
index 92efc3e9445..b7c1e3d003c 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/EmbeddedSolrServerTestRule.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/EmbeddedSolrServerTestRule.java
@@ -100,8 +100,8 @@ public class EmbeddedSolrServerTestRule extends SolrClientTestRule {
   }
 
   @Override
-  public EmbeddedSolrServer getSolrClient(String name) {
-    return new EmbeddedSolrServer(getCoreContainer(), name);
+  public EmbeddedSolrServer getSolrClient(String collection) {
+    return new EmbeddedSolrServer(getCoreContainer(), collection);
   }
 
   public CoreContainer getCoreContainer() {
diff --git a/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java b/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java
index b3e8b3a7e4a..ce066c4e1eb 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java
@@ -56,8 +56,7 @@ public abstract class RestTestBase extends SolrJettyTestBase {
 
     createAndStartJetty(solrHome, configFile, schemaFile, context, stopAtShutdown, extraServlets);
 
-    restTestHarness =
-        new RestTestHarness(() -> jetty.getBaseUrl().toString() + "/" + DEFAULT_TEST_CORENAME);
+    restTestHarness = new RestTestHarness(() -> getCoreUrl());
   }
 
   /** Validates an update XML String is successful */
diff --git a/solr/test-framework/src/java/org/apache/solr/util/SolrClientTestRule.java b/solr/test-framework/src/java/org/apache/solr/util/SolrClientTestRule.java
index 59d212f19cc..a0407093bb4 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/SolrClientTestRule.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/SolrClientTestRule.java
@@ -135,15 +135,23 @@ public abstract class SolrClientTestRule extends ExternalResource {
     req.process(getAdminClient());
   }
 
-  /** Provides a SolrClient instance for administration actions */
-  public abstract SolrClient getAdminClient();
+  /**
+   * Provides a SolrClient instance for administration actions. The caller doesn't need to close it
+   */
+  public SolrClient getAdminClient() {
+    return getSolrClient(null);
+  }
 
-  /** Provides a SolrClient instance for collection1 */
+  /** Provides a SolrClient instance for collection1. The caller doesn't need to close it */
   public SolrClient getSolrClient() {
     return getSolrClient("collection1");
   }
 
-  public abstract SolrClient getSolrClient(String name);
+  /**
+   * Provides a SolrClient instance for caller defined collection name. The caller doesn't need to
+   * close it
+   */
+  public abstract SolrClient getSolrClient(String collection);
 
   public void clearIndex() throws SolrServerException, IOException {
     new UpdateRequest().deleteByQuery("*:*").commit(getSolrClient(), null);
diff --git a/solr/test-framework/src/java/org/apache/solr/util/SolrJettyTestRule.java b/solr/test-framework/src/java/org/apache/solr/util/SolrJettyTestRule.java
new file mode 100644
index 00000000000..a7abba9c852
--- /dev/null
+++ b/solr/test-framework/src/java/org/apache/solr/util/SolrJettyTestRule.java
@@ -0,0 +1,134 @@
+/*
+ * 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.util;
+
+import java.lang.invoke.MethodHandles;
+import java.nio.file.Path;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.embedded.JettyConfig;
+import org.apache.solr.embedded.JettySolrRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link SolrClientTestRule} that provides a Solr instance running in Jetty, an HTTP server. It's
+ * based off of {@link JettySolrRunner}.
+ */
+public class SolrJettyTestRule extends SolrClientTestRule {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private JettySolrRunner jetty;
+
+  private final ConcurrentHashMap<String, SolrClient> clients = new ConcurrentHashMap<>();
+  private boolean enableProxy;
+
+  @Override
+  protected void after() {
+    for (SolrClient solrClient : clients.values()) {
+      IOUtils.closeQuietly(solrClient);
+    }
+    clients.clear();
+
+    if (jetty != null) {
+      try {
+        jetty.stop();
+      } catch (RuntimeException e) {
+        throw e;
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+      jetty = null;
+      enableProxy = false;
+    }
+  }
+
+  /** Resets the state. DEPRECATED; please don't call! */
+  @Deprecated
+  public void reset() {
+    after();
+  }
+
+  @Override
+  public void startSolr(Path solrHome) {
+    startSolr(
+        solrHome,
+        new Properties(),
+        JettyConfig.builder()
+            .withSSLConfig(SolrTestCaseJ4.sslConfig.buildServerSSLConfig())
+            .build());
+  }
+
+  /**
+   * Enables proxy feature to allow for failure injection testing at the inter-node communication
+   * level. Must be called prior to starting.
+   *
+   * @see JettySolrRunner#getProxy()
+   */
+  public void enableProxy() {
+    assert jetty == null;
+    this.enableProxy = true;
+  }
+
+  public void startSolr(Path solrHome, Properties nodeProperties, JettyConfig jettyConfig) {
+    if (jetty != null) throw new IllegalStateException("Jetty is already running");
+
+    jetty = new JettySolrRunner(solrHome.toString(), nodeProperties, jettyConfig, enableProxy);
+    try {
+      jetty.start();
+    } catch (RuntimeException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    int port = jetty.getLocalPort();
+    log.info("Jetty Assigned Port#{}", port);
+  }
+
+  public JettySolrRunner getJetty() {
+    if (jetty == null) throw new IllegalStateException("Jetty has not started");
+    return jetty;
+  }
+
+  @Override
+  public SolrClient getSolrClient(String collection) {
+    if (collection == null) {
+      collection = "";
+    }
+    return clients.computeIfAbsent(collection, this::newSolrClient);
+  }
+
+  protected SolrClient newSolrClient(String collection) {
+    String url = getBaseUrl() + (StrUtils.isBlank(collection) ? "" : "/" + collection);
+    return new HttpSolrClient.Builder(url).build();
+  }
+
+  /** URL to Solr. */
+  public String getBaseUrl() {
+    return getJetty().getBaseUrl().toString();
+  }
+
+  public CoreContainer getCoreContainer() {
+    return getJetty().getCoreContainer();
+  }
+}