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

(solr) branch branch_9x updated: SOLR-17066: Switch ConcurrentSolrClients away from core URLs (#2254)

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

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


The following commit(s) were added to refs/heads/branch_9x by this push:
     new a406bdd2982 SOLR-17066: Switch ConcurrentSolrClients away from core URLs (#2254)
a406bdd2982 is described below

commit a406bdd29823717cd445928ad884da62d383c8c8
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Wed Feb 14 12:15:52 2024 -0500

    SOLR-17066: Switch ConcurrentSolrClients away from core URLs (#2254)
    
    Providing a core URL as a SolrClient's "base URL" prevents it from
    communicating with other cores or making core-agnostic API requests
    (e.g. node healthcheck, list cores, etc.)
    
    This commit migrates all usage of both "concurrent update" clients
    away from core URLs.
---
 .../src/java/org/apache/solr/update/StreamingSolrClients.java     | 7 +++++++
 .../org/apache/solr/client/solrj/impl/ConnectionReuseTest.java    | 3 ++-
 .../test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java  | 4 ++--
 .../java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java  | 3 ---
 .../solrj/embedded/SolrExampleStreamingBinaryHttp2Test.java       | 3 ++-
 .../client/solrj/embedded/SolrExampleStreamingBinaryTest.java     | 3 ++-
 .../solr/client/solrj/embedded/SolrExampleStreamingHttp2Test.java | 6 ++++--
 .../solr/client/solrj/embedded/SolrExampleStreamingTest.java      | 3 ++-
 .../solrj/impl/ConcurrentUpdateHttp2SolrClientBadInputTest.java   | 8 ++++----
 .../client/solrj/impl/ConcurrentUpdateHttp2SolrClientTest.java    | 6 ++++--
 .../client/solrj/impl/ConcurrentUpdateSolrClientBadInputTest.java | 3 ++-
 .../client/solrj/impl/ConcurrentUpdateSolrClientBuilderTest.java  | 6 ++++--
 .../solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java    | 6 ++++--
 .../apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java  | 1 +
 .../apache/solr/cloud/FullThrottleStoppableIndexingThread.java    | 2 ++
 15 files changed, 42 insertions(+), 22 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/update/StreamingSolrClients.java b/solr/core/src/java/org/apache/solr/update/StreamingSolrClients.java
index e53797ee21f..51381f4a582 100644
--- a/solr/core/src/java/org/apache/solr/update/StreamingSolrClients.java
+++ b/solr/core/src/java/org/apache/solr/update/StreamingSolrClients.java
@@ -158,6 +158,13 @@ class ErrorReportingConcurrentUpdateSolrClient extends ConcurrentUpdateHttp2Solr
     protected SolrCmdDistributor.Req req;
     protected List<SolrError> errors;
 
+    /**
+     * @param baseSolrUrl the base URL of a Solr node. Should <em>not</em> contain a collection or
+     *     core name
+     * @param client the client to use in making requests
+     * @param req the command distributor request object for this client
+     * @param errors a collector for any errors
+     */
     public Builder(
         String baseSolrUrl,
         Http2SolrClient client,
diff --git a/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java b/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
index 20965ef6cb6..b10f72152e1 100644
--- a/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
+++ b/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
@@ -83,7 +83,8 @@ public class ConnectionReuseTest extends SolrCloudTestCase {
     switch (random().nextInt(3)) {
       case 0:
         // currently, only testing with 1 thread
-        return new ConcurrentUpdateSolrClient.Builder(url.toString() + "/" + COLLECTION)
+        return new ConcurrentUpdateSolrClient.Builder(url.toString())
+            .withDefaultCollection(COLLECTION)
             .withHttpClient(httpClient)
             .withQueueSize(6)
             .withThreadCount(1)
diff --git a/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java b/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
index 5f0a86b0822..318a28a289f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
@@ -812,8 +812,8 @@ public class FullSolrCloudDistribCmdsTest extends SolrCloudTestCase {
     final int numDocs = atLeast(50);
     final JettySolrRunner nodeToUpdate = cluster.getRandomJetty(random());
     try (ConcurrentUpdateSolrClient indexClient =
-        new ConcurrentUpdateSolrClient.Builder(
-                nodeToUpdate.getProxyBaseUrl() + "/" + collectionName)
+        new ConcurrentUpdateSolrClient.Builder(nodeToUpdate.getProxyBaseUrl().toString())
+            .withDefaultCollection(collectionName)
             .withQueueSize(10)
             .withThreadCount(2)
             .build()) {
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
index 748b48abeec..b595701aec5 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
@@ -144,9 +144,6 @@ public class LBHttpSolrClient extends LBSolrClient {
               .withResponseParser(parser)
               .withConnectionTimeout(connectionTimeoutMillis, TimeUnit.MILLISECONDS)
               .withSocketTimeout(soTimeoutMillis, TimeUnit.MILLISECONDS);
-
-      // Note that this may override or even conflict with a value in the base URL ... should we log
-      // a warning here?
       if (defaultCollection != null) {
         clientBuilder.withDefaultCollection(defaultCollection);
       }
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 cbab498a9ac..f2ca853005f 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 = getCoreUrl();
+    String url = getBaseUrl();
     // smaller queue size hits locks more often
     Http2SolrClient solrClient =
         new Http2SolrClient.Builder()
@@ -46,6 +46,7 @@ public class SolrExampleStreamingBinaryHttp2Test extends SolrExampleStreamingHtt
             .build();
     ConcurrentUpdateHttp2SolrClient concurrentClient =
         new ErrorTrackingConcurrentUpdateSolrClient.Builder(url, solrClient)
+            .withDefaultCollection(DEFAULT_TEST_CORENAME)
             .withQueueSize(2)
             .withThreadCount(5)
             .build();
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 ec326f61858..1610760a84f 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,8 @@ public class SolrExampleStreamingBinaryTest extends SolrExampleStreamingTest {
   public SolrClient createNewSolrClient() {
 
     SolrClient client =
-        new ErrorTrackingConcurrentUpdateSolrClient.Builder(getCoreUrl())
+        new ErrorTrackingConcurrentUpdateSolrClient.Builder(getBaseUrl())
+            .withDefaultCollection(DEFAULT_TEST_CORENAME)
             .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 1584fc87cf1..7c20c53254f 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 = getCoreUrl();
+    String url = getBaseUrl();
     // smaller queue size hits locks more often
     Http2SolrClient solrClient =
         new Http2SolrClient.Builder()
@@ -54,6 +54,7 @@ public class SolrExampleStreamingHttp2Test extends SolrExampleTests {
             .build();
     ConcurrentUpdateHttp2SolrClient concurrentClient =
         new ErrorTrackingConcurrentUpdateSolrClient.Builder(url, solrClient)
+            .withDefaultCollection(DEFAULT_TEST_CORENAME)
             .withQueueSize(2)
             .withThreadCount(5)
             .build();
@@ -63,10 +64,11 @@ public class SolrExampleStreamingHttp2Test extends SolrExampleTests {
   public void testWaitOptions() throws Exception {
     // SOLR-3903
     final List<Throwable> failures = new ArrayList<>();
-    final String serverUrl = getCoreUrl();
+    final String serverUrl = getBaseUrl();
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         ConcurrentUpdateHttp2SolrClient concurrentClient =
             new FailureRecordingConcurrentUpdateSolrClient.Builder(serverUrl, http2Client)
+                .withDefaultCollection(DEFAULT_TEST_CORENAME)
                 .withQueueSize(2)
                 .withThreadCount(2)
                 .build()) {
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 6aa438e0339..616782a10ea 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,8 @@ public class SolrExampleStreamingTest extends SolrExampleTests {
   @Override
   public SolrClient createNewSolrClient() {
     // smaller queue size hits locks more often
-    return new ErrorTrackingConcurrentUpdateSolrClient.Builder(getCoreUrl())
+    return new ErrorTrackingConcurrentUpdateSolrClient.Builder(getBaseUrl())
+        .withDefaultCollection(DEFAULT_TEST_CORENAME)
         .withQueueSize(2)
         .withThreadCount(5)
         .withResponseParser(new XMLResponseParser())
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 552a48c379b..8ca9d6b938d 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
@@ -45,8 +45,8 @@ public class ConcurrentUpdateHttp2SolrClientBadInputTest extends SolrJettyTestBa
 
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         SolrClient client =
-            new ConcurrentUpdateHttp2SolrClient.Builder(
-                    getBaseUrl() + "/" + ANY_COLLECTION, http2Client)
+            new ConcurrentUpdateHttp2SolrClient.Builder(getBaseUrl(), http2Client)
+                .withDefaultCollection(ANY_COLLECTION)
                 .withQueueSize(ANY_QUEUE_SIZE)
                 .withThreadCount(ANY_MAX_NUM_THREADS)
                 .build()) {
@@ -78,8 +78,8 @@ public class ConcurrentUpdateHttp2SolrClientBadInputTest extends SolrJettyTestBa
 
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         SolrClient client =
-            new ConcurrentUpdateHttp2SolrClient.Builder(
-                    getBaseUrl() + "/" + ANY_COLLECTION, http2Client)
+            new ConcurrentUpdateHttp2SolrClient.Builder(getBaseUrl(), http2Client)
+                .withDefaultCollection(ANY_COLLECTION)
                 .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 70804ff2654..713ee22a926 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
@@ -146,7 +146,8 @@ public class ConcurrentUpdateHttp2SolrClientTest extends SolrJettyTestBase {
 
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         ConcurrentUpdateHttp2SolrClient concurrentClient =
-            new ConcurrentUpdateHttp2SolrClient.Builder(getCoreUrl(), http2Client)
+            new ConcurrentUpdateHttp2SolrClient.Builder(getBaseUrl(), http2Client)
+                .withDefaultCollection(DEFAULT_TEST_CORENAME)
                 .withQueueSize(cussQueueSize)
                 .withThreadCount(cussThreadCount)
                 .build()) {
@@ -205,7 +206,8 @@ public class ConcurrentUpdateHttp2SolrClientTest extends SolrJettyTestBase {
 
     try (Http2SolrClient http2Client = new Http2SolrClient.Builder().build();
         ConcurrentUpdateHttp2SolrClient concurrentClient =
-            new ConcurrentUpdateHttp2SolrClient.Builder(getCoreUrl(), http2Client)
+            new ConcurrentUpdateHttp2SolrClient.Builder(getBaseUrl(), http2Client)
+                .withDefaultCollection(DEFAULT_TEST_CORENAME)
                 .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 17e74924a47..45520dd49c4 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,8 @@ public class ConcurrentUpdateSolrClientBadInputTest extends SolrJettyTestBase {
   @Test
   public void testDeleteByIdReportsInvalidIdLists() throws Exception {
     try (SolrClient client =
-        new ConcurrentUpdateSolrClient.Builder(getBaseUrl() + "/" + ANY_COLLECTION)
+        new ConcurrentUpdateSolrClient.Builder(getBaseUrl())
+            .withDefaultCollection(ANY_COLLECTION)
             .withQueueSize(ANY_QUEUE_SIZE)
             .withThreadCount(ANY_MAX_NUM_THREADS)
             .build()) {
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBuilderTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBuilderTest.java
index c6cd4779f96..d2c5686d6bb 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBuilderTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientBuilderTest.java
@@ -31,6 +31,8 @@ import org.junit.Test;
 /** Unit tests for {@link Builder}. */
 public class ConcurrentUpdateSolrClientBuilderTest extends SolrTestCase {
 
+  private static final String ANY_BASE_URL = "http://localhost:8983/solr";
+
   @Test(expected = IllegalArgumentException.class)
   public void testRejectsMissingBaseSolrUrl() {
     new Builder(null).build();
@@ -39,7 +41,7 @@ public class ConcurrentUpdateSolrClientBuilderTest extends SolrTestCase {
   @Test
   @SuppressWarnings({"try"})
   public void testMissingQueueSize() {
-    try (ConcurrentUpdateSolrClient client = new Builder("someurl").build()) {
+    try (ConcurrentUpdateSolrClient client = new Builder(ANY_BASE_URL).build()) {
       // Do nothing as we just need to test that the only mandatory parameter for building the
       // client is the baseSolrUrl
     }
@@ -76,7 +78,7 @@ public class ConcurrentUpdateSolrClientBuilderTest extends SolrTestCase {
   @Test
   public void testDefaultCollectionPassedFromBuilderToClient() throws IOException {
     try (SolrClient createdClient =
-        new ConcurrentUpdateSolrClient.Builder("someurl")
+        new ConcurrentUpdateSolrClient.Builder(ANY_BASE_URL)
             .withDefaultCollection("aCollection")
             .build()) {
       assertEquals("aCollection", createdClient.getDefaultCollection());
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 6cff2257f8d..28052321a69 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
@@ -227,7 +227,8 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
     }
 
     try (ConcurrentUpdateSolrClient concurrentClient =
-        (new ConcurrentUpdateSolrClient.Builder(getCoreUrl()))
+        (new ConcurrentUpdateSolrClient.Builder(getBaseUrl()))
+            .withDefaultCollection(DEFAULT_TEST_CORENAME)
             .withQueueSize(cussQueueSize)
             .withThreadCount(cussThreadCount)
             .build()) {
@@ -283,7 +284,8 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
     }
 
     try (ConcurrentUpdateSolrClient concurrentClient =
-        (new ConcurrentUpdateSolrClient.Builder(getCoreUrl()))
+        (new ConcurrentUpdateSolrClient.Builder(getBaseUrl()))
+            .withDefaultCollection(DEFAULT_TEST_CORENAME)
             .withQueueSize(cussQueueSize)
             .withThreadCount(cussThreadCount)
             .build()) {
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 c169faab0e7..e3c589dae6b 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
@@ -49,6 +49,7 @@ public class HttpSolrClientConPoolTest extends SolrJettyTestBase {
   public static void beforeTest() throws Exception {
     createAndStartJetty(legacyExampleCollection1SolrHome());
     fooUrl = getBaseUrl();
+
     secondJetty.startSolr(Path.of(legacyExampleCollection1SolrHome()));
     barUrl = secondJetty.getBaseUrl();
   }
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java b/solr/test-framework/src/java/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java
index 79413ed51e8..23846f0fff2 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/FullThrottleStoppableIndexingThread.java
@@ -63,6 +63,7 @@ class FullThrottleStoppableIndexingThread extends StoppableIndexingThread {
     cusc =
         new ErrorLoggingConcurrentUpdateSolrClient.Builder(
                 ((HttpSolrClient) clients.get(0)).getBaseURL())
+            .withDefaultCollection(clients.get(0).getDefaultCollection())
             .withHttpClient(httpClient)
             .withQueueSize(8)
             .withThreadCount(2)
@@ -127,6 +128,7 @@ class FullThrottleStoppableIndexingThread extends StoppableIndexingThread {
       cusc =
           new ErrorLoggingConcurrentUpdateSolrClient.Builder(
                   ((HttpSolrClient) clients.get(clientIndex)).getBaseURL())
+              .withDefaultCollection(clients.get(clientIndex).getDefaultCollection())
               .withHttpClient(httpClient)
               .withQueueSize(30)
               .withThreadCount(3)