You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ho...@apache.org on 2023/01/07 00:54:09 UTC

[solr] branch jira/SOLR-6312 created (now 029260f9888)

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

hossman pushed a change to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git


      at 029260f9888 Merge branch 'main' into jira/SOLR-6312

This branch includes the following new commits:

     new b4edbd3d172 refactor TrackingUpdateProcessorFactory (and RecordingUpdateProcessorFactory) into test-framework so they can be re-used in solrj tests
     new 6986e9abfd0 Flesh out test of the bug showing the expected behavior
     new cb52dd264e5 Fix root cause of bug
     new 52fe0227203 Fix tests that expect updates routed to leaders to explicitly configure sendUpdatesOnlyToShardLeaders() on their randomized client Builders
     new 5c65eb714b5 remove unused variable that confuses the code
     new 1ad2a161b78 Beef up the assertions we can make by refactoring RecordingResults to track requests and commands (ot just core names)
     new 8b998c5b619 Beef up testing to include multi direct updates in a single request
     new 345afd12fa2 Add IsUpdateRequest.isSendToLeaders() such that both it and client config must be true for shards.preference to be overridden
     new e9c4f80659a Fix Builder method name and update javadocs
     new a27567f8a92 Add CloudHttp2SolrClient support and testing, fill in test class jdocs
     new 963f1ea3864 resolve obnoxious forbidden API
     new 029260f9888 Merge branch 'main' into jira/SOLR-6312

The 12 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[solr] 07/12: Beef up testing to include multi direct updates in a single request

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit 8b998c5b6197b54483683da8ed4a7a15253ab394
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Tue Dec 20 17:48:24 2022 -0700

    Beef up testing to include multi direct updates in a single request
---
 .../impl/SendUpdatesToLeadersOverrideTest.java     | 77 +++++++++++++++++++---
 1 file changed, 67 insertions(+), 10 deletions(-)

diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
index 1e72d6f16f1..0daaa76dcb0 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
@@ -219,7 +219,8 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
     try (CloudSolrClient client =
         new CloudLegacySolrClient.Builder(
                 Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
-            // nocommit: builder name should never have been 'All', should have been 'Any'
+            // nocommit: builder name should never have been 'sendUpdatesToAllReplicasInShard'
+            // nocommit: - should have been 'sendUpdatesToAnyReplicas'
             .sendUpdatesToAllReplicasInShard()
             .build()) {
       checkUpdatesWithShardsPrefPull(client);
@@ -300,11 +301,35 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
     { // When we have multiple direct updates for different shards, client will
       // split them and merge the responses.
       //
-      // But we should still only see at most one request per shard leader (pre and post)
+      // But we should still only see at most one pre request per shard leader
 
-      // nocommit: check an UpdateRequest with a mix of many doc adds DBIs, ...
-      // nocommit: confirm no more then 2 pre-requests, one to each leader
-      // nocommit: confirm same pre and post requests
+      final RecordingResults record =
+          assertUpdateWithRecording(prefPull(createMultiDirectUpdates(100, 10)), client);
+
+      // NOTE: Don't assume our docIds are spread across multi-shards...
+      // ...but the original number of requests should all be diff leaders
+      assertThat(
+          "multi pre-distrib must be leaders",
+          record.preDistribCores.keySet(),
+          everyItem(isIn(LEADER_CORE_NAMES)));
+      assertThat(
+          "multi pre-distrib req != pre-distrib num cores",
+          record.preDistribRequests.keySet(),
+          hasSize(record.preDistribCores.keySet().size()));
+      assertThat("multi pre-distrib command size", record.preDistribCommands, hasSize(100 + 10));
+
+      assertEquals(
+          "multi post-distrib must be same leaders",
+          record.preDistribCores.keySet(),
+          record.postDistribCores.keySet());
+
+      // NOTE: we make no asertion about number of post-distrb requests, just commands
+      // (distrib proc may batch differently then what we send)
+      assertEquals(
+          "multi post-distrib cores don't match pre-distrib cores",
+          record.preDistribCores.keySet(),
+          record.postDistribCores.keySet());
+      assertThat("multi post-distrib command size", record.postDistribCommands, hasSize(100 + 10));
     }
   }
 
@@ -385,13 +410,45 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
           "dbq post-distrib size", record.postDistribCommands, hasSize(LEADER_CORE_NAMES.size()));
     }
 
-    { // When we have multiple direct updates for different shards, client will
-      // split them and merge the responses.
+    { // When we sendToLeaders is disabled, a single UpdateRequest containing multiple adds
+      // should still only go to one replica for all the "pre" commands, then be forwarded
+      // the respective leaders for the "post" commands
+
+      final RecordingResults record =
+          assertUpdateWithRecording(prefPull(createMultiDirectUpdates(100, 10)), client);
+
+      assertThat("multi pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
+      assertThat(
+          "multi pre-distrib must be PULL",
+          record.preDistribCores.keySet(),
+          everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
+      assertThat("multi pre-distrib req size", record.preDistribRequests.keySet(), hasSize(1));
+      assertThat("multi pre-distrib command size", record.preDistribCommands, hasSize(100 + 10));
+
+      assertEquals(
+          "multi post-distrib must be all leaders",
+          LEADER_CORE_NAMES,
+          record.postDistribCores.keySet());
+      // NOTE: Don't assume our docIds are spread across multi-shards...
       //
-      // But we should still only see at most one request per shard leader (pre and post)
+      // We make no asertion about number of post-distrb requests
+      // (distrib proc may batch differently then what we send)
+      assertThat(
+          "multi post-distrib cores",
+          record.postDistribCores.keySet(),
+          everyItem(isIn(LEADER_CORE_NAMES)));
+      assertThat("multi post-distrib command size", record.postDistribCommands, hasSize(100 + 10));
+    }
+  }
 
-      // nocommit: check an UpdateRequest with a mix of many doc adds DBIs, ...
-      // nocommit: confirm we get exactly one pre request, no request splitting kicking in
+  private static UpdateRequest createMultiDirectUpdates(final int numAdds, final int numDel) {
+    final UpdateRequest req = new UpdateRequest();
+    for (int i = 0; i < numAdds; i++) {
+      req.add(sdoc("id", "add" + i));
     }
+    for (int i = 0; i < numDel; i++) {
+      req.deleteById("del" + i);
+    }
+    return req;
   }
 }


[solr] 05/12: remove unused variable that confuses the code

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit 5c65eb714b520754957fe60e099c94379a09c23d
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Tue Dec 20 09:28:04 2022 -0700

    remove unused variable that confuses the code
---
 .../java/org/apache/solr/client/solrj/impl/CloudSolrClient.java  | 9 +++------
 1 file changed, 3 insertions(+), 6 deletions(-)

diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index 56a9c4e14c4..c05fcf70b70 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -842,11 +842,10 @@ public abstract class CloudSolrClient extends SolrClient {
       isCollectionRequestOfV2 = ((V2Request) request).isPerCollectionRequest();
     }
     boolean isAdmin = ADMIN_PATHS.contains(request.getPath());
-    boolean isUpdate = (request instanceof IsUpdateRequest) && (request instanceof UpdateRequest);
     if (!inputCollections.isEmpty()
         && !isAdmin
         && !isCollectionRequestOfV2) { // don't do _stateVer_ checking for admin, v2 api requests
-      Set<String> requestedCollectionNames = resolveAliases(inputCollections, isUpdate);
+      Set<String> requestedCollectionNames = resolveAliases(inputCollections);
 
       StringBuilder stateVerParamBuilder = null;
       for (String requestedCollection : requestedCollectionNames) {
@@ -1043,7 +1042,6 @@ public abstract class CloudSolrClient extends SolrClient {
     connect();
 
     boolean sendToLeaders = false;
-    boolean isUpdate = false;
 
     if (request instanceof IsUpdateRequest) {
       sendToLeaders =
@@ -1052,7 +1050,6 @@ public abstract class CloudSolrClient extends SolrClient {
 
       // Check if we can do a "directUpdate" ...
       if (sendToLeaders && request instanceof UpdateRequest) {
-        isUpdate = true;
         if (inputCollections.size() > 1) {
           throw new SolrException(
               SolrException.ErrorCode.BAD_REQUEST,
@@ -1098,7 +1095,7 @@ public abstract class CloudSolrClient extends SolrClient {
       }
 
     } else { // Typical...
-      Set<String> collectionNames = resolveAliases(inputCollections, isUpdate);
+      Set<String> collectionNames = resolveAliases(inputCollections);
       if (collectionNames.isEmpty()) {
         throw new SolrException(
             SolrException.ErrorCode.BAD_REQUEST,
@@ -1193,7 +1190,7 @@ public abstract class CloudSolrClient extends SolrClient {
    * Resolves the input collections to their possible aliased collections. Doesn't validate
    * collection existence.
    */
-  private Set<String> resolveAliases(List<String> inputCollections, boolean isUpdate) {
+  private Set<String> resolveAliases(List<String> inputCollections) {
     if (inputCollections.isEmpty()) {
       return Collections.emptySet();
     }


[solr] 02/12: Flesh out test of the bug showing the expected behavior

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit 6986e9abfd0ec6727957828a61c09faa63a2ff5f
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Fri Dec 16 14:50:36 2022 -0700

    Flesh out test of the bug showing the expected behavior
---
 .../configsets/tracking-updates/conf/schema.xml    |  29 ++
 .../tracking-updates/conf/solrconfig.xml           |  63 ++++
 .../impl/SendUpdatesToLeadersOverrideTest.java     | 316 +++++++++++++++++++++
 3 files changed, 408 insertions(+)

diff --git a/solr/solrj/src/test-files/solrj/solr/configsets/tracking-updates/conf/schema.xml b/solr/solrj/src/test-files/solrj/solr/configsets/tracking-updates/conf/schema.xml
new file mode 100644
index 00000000000..4124feab0c3
--- /dev/null
+++ b/solr/solrj/src/test-files/solrj/solr/configsets/tracking-updates/conf/schema.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<schema name="minimal" version="1.1">
+  <fieldType name="string" class="solr.StrField"/>
+  <fieldType name="int" class="${solr.tests.IntegerFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <dynamicField name="*" type="string" indexed="true" stored="true"/>
+  <!-- for versioning -->
+  <field name="_version_" type="long" indexed="true" stored="true"/>
+  <field name="_root_" type="string" indexed="true" stored="true" multiValued="false" required="false"/>
+  <field name="id" type="string" indexed="true" stored="true"/>
+  <dynamicField name="*_s"  type="string"  indexed="true"  stored="true" />
+  <uniqueKey>id</uniqueKey>
+</schema>
diff --git a/solr/solrj/src/test-files/solrj/solr/configsets/tracking-updates/conf/solrconfig.xml b/solr/solrj/src/test-files/solrj/solr/configsets/tracking-updates/conf/solrconfig.xml
new file mode 100644
index 00000000000..d6df57ba134
--- /dev/null
+++ b/solr/solrj/src/test-files/solrj/solr/configsets/tracking-updates/conf/solrconfig.xml
@@ -0,0 +1,63 @@
+<?xml version="1.0" ?>
+
+<!--
+ 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.
+-->
+
+<!-- Minimal solrconfig.xml with tracking enabled on the URP, both before and after distrib -->
+
+<config>
+
+  <updateRequestProcessorChain default="true">
+    <processor class="solr.LogUpdateProcessorFactory" />
+    <processor class="solr.TrackingUpdateProcessorFactory">
+      <str name="group">pre-distrib</str>
+    </processor>
+    <processor class="solr.DistributedUpdateProcessorFactory" />
+    <processor class="solr.TrackingUpdateProcessorFactory">
+      <str name="group">post-distrib</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+  
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <commitWithin>
+      <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
+    </commitWithin>
+    <updateLog class="${solr.ulog:solr.UpdateLog}"></updateLog>
+  </updateHandler>
+
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+
+  </requestHandler>
+  <indexConfig>
+    <mergeScheduler class="${solr.mscheduler:org.apache.lucene.index.ConcurrentMergeScheduler}"/>
+  </indexConfig>
+</config>
+
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
new file mode 100644
index 00000000000..af7d86a649b
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.impl;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.isIn;
+import static org.hamcrest.Matchers.not;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.update.processor.TrackingUpdateProcessorFactory;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test the behavior of {@link CloudSolrClient#isUpdatesToLeaders}
+ *
+ * <p>nocommit: more explanation of how we test this
+ */
+public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String CONFIG = "tracking-updates";
+  private static final String COLLECTION_NAME = "the_collection";
+
+  private static final Set<String> LEADER_CORE_NAMES = new HashSet<>();
+  private static final Set<String> PULL_REPLICA_CORE_NAMES = new HashSet<>();
+
+  @AfterClass
+  public static void cleanupExpectedCoreNames() throws Exception {
+    LEADER_CORE_NAMES.clear();
+    PULL_REPLICA_CORE_NAMES.clear();
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    assert LEADER_CORE_NAMES.isEmpty();
+    assert PULL_REPLICA_CORE_NAMES.isEmpty();
+
+    final int numNodes = 4;
+    configureCluster(numNodes)
+        .addConfig(
+            CONFIG,
+            getFile("solrj")
+                .toPath()
+                .resolve("solr")
+                .resolve("configsets")
+                .resolve(CONFIG)
+                .resolve("conf"))
+        .configure();
+
+    // create 2 shard collection with 1 NRT (leader) and 1 PULL replica
+    assertTrue(
+        CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG, 2, 1)
+            .setPullReplicas(1)
+            .setNrtReplicas(1)
+            .process(cluster.getSolrClient())
+            .isSuccess());
+
+    final List<Replica> allReplicas =
+        cluster.getSolrClient().getClusterState().getCollection(COLLECTION_NAME).getReplicas();
+    assertEquals(
+        "test preconditions were broken, each replica should have it's own node",
+        numNodes,
+        allReplicas.size());
+
+    allReplicas.stream()
+        .filter(Replica::isLeader)
+        .map(Replica::getCoreName)
+        .collect(Collectors.toCollection(() -> LEADER_CORE_NAMES));
+
+    allReplicas.stream()
+        .filter(r -> Replica.Type.PULL.equals(r.getType()))
+        .map(Replica::getCoreName)
+        .collect(Collectors.toCollection(() -> PULL_REPLICA_CORE_NAMES));
+
+    log.info("Leader coreNames={}", LEADER_CORE_NAMES);
+    log.info("PULL Replica coreNames={}", PULL_REPLICA_CORE_NAMES);
+  }
+
+  /**
+   * Helper that stops recording and returns an unmodifiable list of the core names from each
+   * recorded command
+   */
+  private static List<String> stopRecording(final String group) {
+    return TrackingUpdateProcessorFactory.stopRecording(group).stream()
+        .map(
+            uc ->
+                uc.getReq()
+                    .getContext()
+                    .get(TrackingUpdateProcessorFactory.REQUEST_NODE)
+                    .toString())
+        .collect(Collectors.toUnmodifiableList());
+  }
+
+  private static class RecordingResults {
+    public final List<String> preDistribCoreNames;
+    public final List<String> postDistribCoreNames;
+
+    public RecordingResults(
+        final List<String> preDistribCoreNames, final List<String> postDistribCoreNames) {
+      this.preDistribCoreNames = preDistribCoreNames;
+      this.postDistribCoreNames = postDistribCoreNames;
+    }
+  }
+
+  /**
+   * Given an {@link UpdateRequest} and a {@link SolrClient}, processes that request against that
+   * client while {@link TrackingUpdateProcessorFactory} is recording, does some basic validation,
+   * then passes the recorded <code>pre-distrib</code> and <code>post-distrib</code> coreNames to
+   * the specified validators
+   */
+  private static RecordingResults assertUpdateWithRecording(
+      final UpdateRequest req, final SolrClient client) throws Exception {
+
+    TrackingUpdateProcessorFactory.startRecording("pre-distrib");
+    TrackingUpdateProcessorFactory.startRecording("post-distrib");
+
+    assertEquals(0, req.process(client, COLLECTION_NAME).getStatus());
+
+    final RecordingResults results =
+        new RecordingResults(stopRecording("pre-distrib"), stopRecording("post-distrib"));
+
+    // post-distrib should never match any PULL replicas, regardless of request, if this fails
+    // something is seriously wrong with our cluster
+    assertThat(
+        "post-distrib should never be PULL replica",
+        results.postDistribCoreNames,
+        everyItem(not(isIn(PULL_REPLICA_CORE_NAMES))));
+
+    return results;
+  }
+
+  /**
+   * Since {@link UpdateRequest#setParam} isn't a fluent API, this is a wrapper helper for setting
+   * <code>shards.preference=replica.type:PULL</code> on the input req, and then returning that req
+   */
+  private static UpdateRequest prefPull(final UpdateRequest req) {
+    req.setParam("shards.preference", "replica.type:PULL");
+    return req;
+  }
+
+  // nocommit: - test CloudHttp2SolrClient as well
+
+  // basic sanity check of expected default behavior
+  public void testUpdatesDefaultToLeaders() throws Exception {
+    try (CloudSolrClient client =
+        new CloudLegacySolrClient.Builder(
+                Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            .sendUpdatesOnlyToShardLeaders()
+            .build()) {
+      checkUpdatesDefaultToLeaders(client);
+    }
+  }
+
+  /** nocommit: This test will fail until bug is fixed */
+  public void testUpdatesWithShardsPrefPull() throws Exception {
+    try (CloudSolrClient client =
+        new CloudLegacySolrClient.Builder(
+                Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            // nocommit: builder name should never have been 'All', should have been 'Any'
+            .sendUpdatesToAllReplicasInShard()
+            .build()) {
+      checkUpdatesWithShardsPrefPull(client);
+    }
+  }
+
+  /**
+   * Given a SolrClient, sends various updates and asserts expecations regarding default behavior:
+   * that these requests will be initially sent to shard leaders, and "routed" requests will be sent
+   * to the leader for that route's shard
+   */
+  private void checkUpdatesDefaultToLeaders(final CloudSolrClient client) throws Exception {
+    assertTrue(
+        "broken test, only valid on clients where updatesToLeaders=true",
+        client.isUpdatesToLeaders());
+
+    { // single doc add is routable and should go to a single shard
+      final RecordingResults add =
+          assertUpdateWithRecording(new UpdateRequest().add(sdoc("id", "hoss")), client);
+
+      // single NRT leader is only core that should be involved at all
+      assertThat("add pre-distrib size", add.preDistribCoreNames, hasSize(1));
+      assertThat(
+          "add pre-distrib must be leader",
+          add.preDistribCoreNames,
+          everyItem(isIn(LEADER_CORE_NAMES)));
+      assertEquals(
+          "add pre and post should match", add.preDistribCoreNames, add.postDistribCoreNames);
+
+      // whatever leader our add was routed to, a DBI for the same id should go to the same leader
+      final RecordingResults del =
+          assertUpdateWithRecording(new UpdateRequest().deleteById("hoss"), client);
+      assertEquals(
+          "del pre and post should match", add.preDistribCoreNames, add.postDistribCoreNames);
+      assertEquals(
+          "add and del should have been routed the same",
+          add.preDistribCoreNames,
+          del.preDistribCoreNames);
+    }
+
+    { // DBQ should start on some leader, and then distrib to both leaders
+      final RecordingResults record =
+          assertUpdateWithRecording(new UpdateRequest().deleteByQuery("*:*"), client);
+
+      assertThat("dbq pre-distrib size", record.preDistribCoreNames, hasSize(1));
+      assertThat(
+          "dbq pre-distrib must be leader",
+          record.preDistribCoreNames,
+          everyItem(isIn(LEADER_CORE_NAMES)));
+
+      assertEquals(
+          "dbq post-distrib must be all leaders",
+          LEADER_CORE_NAMES,
+          new HashSet<>(record.postDistribCoreNames));
+    }
+
+    // nocommit: check an UpdateRequest with a mix of many doc adds DBIs, ...
+    // nocommit: confirm exactly 2 requests, one to each leader
+
+  }
+
+  /**
+   * Given a SolrClient, sends various updates using {#link #prefPull} and asserts expecations that
+   * these requests will be initially sent to PULL replcias
+   */
+  private void checkUpdatesWithShardsPrefPull(final CloudSolrClient client) throws Exception {
+
+    assertFalse(
+        "broken test, only valid on clients where updatesToLeaders=false",
+        client.isUpdatesToLeaders());
+
+    { // single doc add...
+      final RecordingResults add =
+          assertUpdateWithRecording(prefPull(new UpdateRequest().add(sdoc("id", "hoss"))), client);
+
+      // ...should start on (some) PULL replica, since we asked nicely
+      assertThat("add pre-distrib size", add.preDistribCoreNames, hasSize(1));
+      assertThat(
+          "add pre-distrib must be PULL",
+          add.preDistribCoreNames,
+          everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
+
+      // ...then be routed to single leader for this id
+      assertThat("add post-distrib size", add.postDistribCoreNames, hasSize(1));
+      assertThat(
+          "add post-distrib must be leader",
+          add.postDistribCoreNames,
+          everyItem(isIn(LEADER_CORE_NAMES)));
+
+      // A DBI should also start on (some) PULL replica,  since we asked nicely.
+      //
+      // then it should be distributed to whaever leader our add doc (for the same id) was sent to
+      final RecordingResults del =
+          assertUpdateWithRecording(prefPull(new UpdateRequest().deleteById("hoss")), client);
+      assertThat("del pre-distrib size", del.preDistribCoreNames, hasSize(1));
+      assertThat(
+          "del pre-distrib must be PULL",
+          del.preDistribCoreNames,
+          everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
+      assertEquals(
+          "add and del should have same post-distrib leader",
+          add.postDistribCoreNames,
+          del.postDistribCoreNames);
+    }
+
+    { // DBQ start on (some) PULL replica, since we asked nicely, then be routed to all leaders
+      final RecordingResults record =
+          assertUpdateWithRecording(prefPull(new UpdateRequest().deleteByQuery("*:*")), client);
+
+      assertThat("dbq pre-distrib size", record.preDistribCoreNames, hasSize(1));
+      assertThat(
+          "dbq pre-distrib must be PULL",
+          record.preDistribCoreNames,
+          everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
+
+      assertEquals(
+          "dbq post-distrib must be all leaders",
+          LEADER_CORE_NAMES,
+          new HashSet<>(record.postDistribCoreNames));
+    }
+
+    // nocommit: check an UpdateRequest with a mix of many doc adds DBIs, ...
+    // nocommit: confirm we get exactly one request, no request splitting kicking in
+
+  }
+}


[solr] 08/12: Add IsUpdateRequest.isSendToLeaders() such that both it and client config must be true for shards.preference to be overridden

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit 345afd12fa2d03b795bcc6de0ee1639476e2e683
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Thu Jan 5 15:26:46 2023 -0700

    Add IsUpdateRequest.isSendToLeaders() such that both it and client config must be true for shards.preference to be overridden
---
 .../solr/client/solrj/impl/CloudSolrClient.java    |  24 +++-
 .../solrj/request/AbstractUpdateRequest.java       |  11 ++
 .../solr/client/solrj/request/IsUpdateRequest.java |  15 ++-
 .../impl/SendUpdatesToLeadersOverrideTest.java     | 135 +++++++++++++++++++--
 4 files changed, 170 insertions(+), 15 deletions(-)

diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index c05fcf70b70..900aaa9108a 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -1044,9 +1044,7 @@ public abstract class CloudSolrClient extends SolrClient {
     boolean sendToLeaders = false;
 
     if (request instanceof IsUpdateRequest) {
-      sendToLeaders =
-          // nocommit: also check request.isSendToLeaders() (still to be added)
-          this.isUpdatesToLeaders();
+      sendToLeaders = ((IsUpdateRequest) request).isSendToLeaders() && this.isUpdatesToLeaders();
 
       // Check if we can do a "directUpdate" ...
       if (sendToLeaders && request instanceof UpdateRequest) {
@@ -1206,11 +1204,31 @@ public abstract class CloudSolrClient extends SolrClient {
     return uniqueNames;
   }
 
+  /**
+   * If true, this client has been configured such that it will generally prefer to send {@link
+   * IsUpdateRequest} requests to a shard leader, if and only if {@link
+   * IsUpdateRequest#isSendToLeaders} is also true. If false, then this client has been configured
+   * to obey normal routing preferences when dealing with {@link IsUpdateRequest} requests.
+   *
+   * @see #isDirectUpdatesToLeadersOnly
+   */
   public boolean isUpdatesToLeaders() {
     return updatesToLeaders;
   }
 
   /**
+   * If true, this client has been configured such that "direct updates" will <em>only</em> be sent
+   * to the current leader of the corrisponding shard, and will not be retried with other replicas.
+   * This method has no effect if {@link #isUpdatesToLeaders()} or {@link
+   * IsUpdateRequest#isSendToLeaders} returns false.
+   *
+   * <p>A "direct update" is any update that can be sent directly to a single shard, and does not
+   * need to be broadcast to every shard. (Example: document updates or "delete by id" when using
+   * the default router; non-direct updates are things like commits and "delete by query").
+   *
+   * <p>NOTE: If a single {@link UpdateRequest} contains multiple "direct updates" for different
+   * shards, this client may break the request up and merge th resposes.
+   *
    * @return true if direct updates are sent to shard leaders only
    */
   public boolean isDirectUpdatesToLeadersOnly() {
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/AbstractUpdateRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/AbstractUpdateRequest.java
index 352fd290f59..733502353dd 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/AbstractUpdateRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/AbstractUpdateRequest.java
@@ -159,4 +159,15 @@ public abstract class AbstractUpdateRequest extends SolrRequest<UpdateResponse>
     this.commitWithin = commitWithin;
     return this;
   }
+
+  private boolean sendToLeaders = true;
+
+  public boolean isSendToLeaders() {
+    return sendToLeaders;
+  }
+
+  public AbstractUpdateRequest setSendToLeaders(final boolean sendToLeaders) {
+    this.sendToLeaders = sendToLeaders;
+    return this;
+  }
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/IsUpdateRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/IsUpdateRequest.java
index 5e8c8aff854..54f449e1358 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/IsUpdateRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/IsUpdateRequest.java
@@ -16,5 +16,18 @@
  */
 package org.apache.solr.client.solrj.request;
 
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+
 /** Marker class so that we can determine which requests are updates. */
-public interface IsUpdateRequest {}
+public interface IsUpdateRequest {
+
+  /**
+   * Indicates if clients should make attempts to route this request to a shard leader, overriding
+   * typical client routing preferences for requests. Defaults to true.
+   *
+   * @see CloudSolrClient#isUpdatesToLeaders
+   */
+  default boolean isSendToLeaders() {
+    return true;
+  }
+}
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
index 0daaa76dcb0..044c24f339d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
@@ -32,6 +32,7 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
 import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.cloud.SolrCloudTestCase;
@@ -165,13 +166,13 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
   }
 
   /**
-   * Given an {@link UpdateRequest} and a {@link SolrClient}, processes that request against that
-   * client while {@link TrackingUpdateProcessorFactory} is recording, does some basic validation,
-   * then passes the recorded <code>pre-distrib</code> and <code>post-distrib</code> coreNames to
-   * the specified validators
+   * Given an {@link AbstractUpdateRequest} and a {@link SolrClient}, processes that request against
+   * that client while {@link TrackingUpdateProcessorFactory} is recording, does some basic
+   * validation, then passes the recorded <code>pre-distrib</code> and <code>post-distrib</code>
+   * coreNames to the specified validators
    */
   private static RecordingResults assertUpdateWithRecording(
-      final UpdateRequest req, final SolrClient client) throws Exception {
+      final AbstractUpdateRequest req, final SolrClient client) throws Exception {
 
     TrackingUpdateProcessorFactory.startRecording("pre-distrib");
     TrackingUpdateProcessorFactory.startRecording("post-distrib");
@@ -194,10 +195,11 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
   }
 
   /**
-   * Since {@link UpdateRequest#setParam} isn't a fluent API, this is a wrapper helper for setting
-   * <code>shards.preference=replica.type:PULL</code> on the input req, and then returning that req
+   * Since {@link AbstractUpdateRequest#setParam} isn't a fluent API, this is a wrapper helper for
+   * setting <code>shards.preference=replica.type:PULL</code> on the input req, and then returning
+   * that req
    */
-  private static UpdateRequest prefPull(final UpdateRequest req) {
+  private static AbstractUpdateRequest prefPull(final AbstractUpdateRequest req) {
     req.setParam("shards.preference", "replica.type:PULL");
     return req;
   }
@@ -205,17 +207,18 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
   // nocommit: - test CloudHttp2SolrClient as well
 
   // basic sanity check of expected default behavior
-  public void testUpdatesDefaultToLeaders() throws Exception {
+  public void testClientThatDefaultsToLeaders() throws Exception {
     try (CloudSolrClient client =
         new CloudLegacySolrClient.Builder(
                 Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
             .sendUpdatesOnlyToShardLeaders()
             .build()) {
       checkUpdatesDefaultToLeaders(client);
+      checkUpdatesWithSendToLeadersFalse(client);
     }
   }
 
-  public void testUpdatesWithShardsPrefPull() throws Exception {
+  public void testClientThatDoesNotDefaultToLeaders() throws Exception {
     try (CloudSolrClient client =
         new CloudLegacySolrClient.Builder(
                 Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
@@ -224,6 +227,7 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
             .sendUpdatesToAllReplicasInShard()
             .build()) {
       checkUpdatesWithShardsPrefPull(client);
+      checkUpdatesWithSendToLeadersFalse(client);
     }
   }
 
@@ -334,7 +338,7 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
   }
 
   /**
-   * Given a SolrClient, sends various updates using {#link #prefPull} and asserts expecations that
+   * Given a SolrClient, sends various updates using {@link #prefPull} and asserts expecations that
    * these requests will be initially sent to PULL replcias
    */
   private void checkUpdatesWithShardsPrefPull(final CloudSolrClient client) throws Exception {
@@ -441,6 +445,115 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
     }
   }
 
+  /**
+   * Given a SolrClient, sends various updates were {@link IsUpdateRequest#isSendToLeaders} returns
+   * false, and asserts expectations that requess using {@link #prefPull} are all sent to PULL
+   * replicas, regardless of how the client is configured.
+   */
+  private void checkUpdatesWithSendToLeadersFalse(final CloudSolrClient client) throws Exception {
+    { // single doc add...
+      final RecordingResults add =
+          assertUpdateWithRecording(
+              prefPull(new UpdateRequest().add(sdoc("id", "hoss"))).setSendToLeaders(false),
+              client);
+
+      // ...should start on (some) PULL replica, since we asked nicely
+      assertThat("add pre-distrib size", add.preDistribCores.keySet(), hasSize(1));
+      assertThat(
+          "add pre-distrib must be PULL",
+          add.preDistribCores.keySet(),
+          everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
+      assertThat("add pre-distrib size", add.preDistribRequests.keySet(), hasSize(1));
+      assertThat("add pre-distrib size", add.preDistribCommands, hasSize(1));
+
+      // ...then be routed to single leader for this id
+      assertThat("add post-distrib size", add.postDistribCores.keySet(), hasSize(1));
+      assertThat(
+          "add post-distrib must be leader",
+          add.postDistribCores.keySet(),
+          everyItem(isIn(LEADER_CORE_NAMES)));
+      assertThat("add post-distrib size", add.postDistribRequests.keySet(), hasSize(1));
+      assertThat("add post-distrib size", add.postDistribCommands, hasSize(1));
+
+      // A DBI should also start on (some) PULL replica,  since we asked nicely.
+      //
+      // then it should be distributed to whatever leader our add doc (for the same id) was sent to
+      final RecordingResults del =
+          assertUpdateWithRecording(
+              prefPull(new UpdateRequest().deleteById("hoss")).setSendToLeaders(false), client);
+      assertThat("del pre-distrib size", del.preDistribCores.keySet(), hasSize(1));
+      assertThat(
+          "del pre-distrib must be PULL",
+          del.preDistribCores.keySet(),
+          everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
+      assertThat("del pre-distrib size", del.preDistribRequests.keySet(), hasSize(1));
+      assertThat("del pre-distrib size", del.preDistribCommands, hasSize(1));
+
+      assertEquals(
+          "add and del should have same post-distrib leader",
+          add.postDistribCores.keySet(),
+          del.postDistribCores.keySet());
+      assertThat("del post-distrib size", del.postDistribRequests.keySet(), hasSize(1));
+      assertThat("del post-distrib size", del.postDistribCommands, hasSize(1));
+    }
+
+    { // DBQ start on (some) PULL replica, since we asked nicely, then be routed to all leaders
+      final RecordingResults record =
+          assertUpdateWithRecording(
+              prefPull(new UpdateRequest().deleteByQuery("*:*")).setSendToLeaders(false), client);
+
+      assertThat("dbq pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
+      assertThat(
+          "dbq pre-distrib must be PULL",
+          record.preDistribCores.keySet(),
+          everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
+      assertThat("dbq pre-distrib size", record.preDistribRequests.keySet(), hasSize(1));
+      assertThat("dbq pre-distrib size", record.preDistribCommands, hasSize(1));
+
+      assertEquals(
+          "dbq post-distrib must be all leaders",
+          LEADER_CORE_NAMES,
+          record.postDistribCores.keySet());
+      assertThat(
+          "dbq post-distrib size",
+          record.postDistribRequests.keySet(),
+          hasSize(LEADER_CORE_NAMES.size()));
+      assertThat(
+          "dbq post-distrib size", record.postDistribCommands, hasSize(LEADER_CORE_NAMES.size()));
+    }
+
+    { // When we sendToLeaders is disabled, a single UpdateRequest containing multiple adds
+      // should still only go to one replica for all the "pre" commands, then be forwarded
+      // the respective leaders for the "post" commands
+
+      final RecordingResults record =
+          assertUpdateWithRecording(
+              prefPull(createMultiDirectUpdates(100, 10)).setSendToLeaders(false), client);
+
+      assertThat("multi pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
+      assertThat(
+          "multi pre-distrib must be PULL",
+          record.preDistribCores.keySet(),
+          everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
+      assertThat("multi pre-distrib req size", record.preDistribRequests.keySet(), hasSize(1));
+      assertThat("multi pre-distrib command size", record.preDistribCommands, hasSize(100 + 10));
+
+      assertEquals(
+          "multi post-distrib must be all leaders",
+          LEADER_CORE_NAMES,
+          record.postDistribCores.keySet());
+      // NOTE: Don't assume our docIds are spread across multi-shards...
+      //
+      // We make no asertion about number of post-distrb requests
+      // (distrib proc may batch differently then what we send)
+      assertThat(
+          "multi post-distrib cores",
+          record.postDistribCores.keySet(),
+          everyItem(isIn(LEADER_CORE_NAMES)));
+      assertThat("multi post-distrib command size", record.postDistribCommands, hasSize(100 + 10));
+    }
+  }
+
   private static UpdateRequest createMultiDirectUpdates(final int numAdds, final int numDel) {
     final UpdateRequest req = new UpdateRequest();
     for (int i = 0; i < numAdds; i++) {


[solr] 11/12: resolve obnoxious forbidden API

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit 963f1ea3864a4520c8aab39c26e5fd8e11096ffe
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Fri Jan 6 16:44:23 2023 -0700

    resolve obnoxious forbidden API
---
 .../impl/SendUpdatesToLeadersOverrideTest.java     | 156 ++++++++++++---------
 1 file changed, 87 insertions(+), 69 deletions(-)

diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
index 9a485a483b3..8d4abab9042 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
@@ -41,6 +41,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.update.UpdateCommand;
 import org.apache.solr.update.processor.TrackingUpdateProcessorFactory;
+import org.hamcrest.MatcherAssert;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.slf4j.Logger;
@@ -193,7 +194,7 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
 
     // post-distrib should never match any PULL replicas, regardless of request, if this fails
     // something is seriously wrong with our cluster
-    assertThat(
+    MatcherAssert.assertThat(
         "post-distrib should never be PULL replica",
         results.postDistribCores.keySet(),
         everyItem(not(isIn(PULL_REPLICA_CORE_NAMES))));
@@ -285,10 +286,10 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
           assertUpdateWithRecording(new UpdateRequest().add(sdoc("id", "hoss")), client);
 
       // single NRT leader is only core that should be involved at all
-      assertThat("add pre-distrib size", add.preDistribCores.keySet(), hasSize(1));
-      assertThat("add pre-distrib size", add.preDistribRequests.keySet(), hasSize(1));
-      assertThat("add pre-distrib size", add.preDistribCommands, hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat("add pre-distrib size", add.preDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat("add pre-distrib size", add.preDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("add pre-distrib size", add.preDistribCommands, hasSize(1));
+      MatcherAssert.assertThat(
           "add pre-distrib must be leader",
           add.preDistribCores.keySet(),
           everyItem(isIn(LEADER_CORE_NAMES)));
@@ -313,31 +314,32 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
           "add and del should have been routed the same",
           add.preDistribCores.keySet(),
           del.preDistribCores.keySet());
-      assertThat("del pre-distrib size", del.preDistribRequests.keySet(), hasSize(1));
-      assertThat("del pre-distrib size", del.preDistribCommands, hasSize(1));
+      MatcherAssert.assertThat("del pre-distrib size", del.preDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("del pre-distrib size", del.preDistribCommands, hasSize(1));
     }
 
     { // DBQ should start on some leader, and then distrib to both leaders
       final RecordingResults record =
           assertUpdateWithRecording(new UpdateRequest().deleteByQuery("*:*"), client);
 
-      assertThat("dbq pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat("dbq pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
           "dbq pre-distrib must be leader",
           record.preDistribCores.keySet(),
           everyItem(isIn(LEADER_CORE_NAMES)));
-      assertThat("dbq pre-distrib size", record.preDistribRequests.keySet(), hasSize(1));
-      assertThat("dbq pre-distrib size", record.preDistribCommands, hasSize(1));
+      MatcherAssert.assertThat(
+          "dbq pre-distrib size", record.preDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("dbq pre-distrib size", record.preDistribCommands, hasSize(1));
 
       assertEquals(
           "dbq post-distrib must be all leaders",
           LEADER_CORE_NAMES,
           record.postDistribCores.keySet());
-      assertThat(
+      MatcherAssert.assertThat(
           "dbq post-distrib size",
           record.postDistribRequests.keySet(),
           hasSize(LEADER_CORE_NAMES.size()));
-      assertThat(
+      MatcherAssert.assertThat(
           "dbq post-distrib size", record.postDistribCommands, hasSize(LEADER_CORE_NAMES.size()));
     }
 
@@ -351,15 +353,16 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
 
       // NOTE: Don't assume our docIds are spread across multi-shards...
       // ...but the original number of requests should all be diff leaders
-      assertThat(
+      MatcherAssert.assertThat(
           "multi pre-distrib must be leaders",
           record.preDistribCores.keySet(),
           everyItem(isIn(LEADER_CORE_NAMES)));
-      assertThat(
+      MatcherAssert.assertThat(
           "multi pre-distrib req != pre-distrib num cores",
           record.preDistribRequests.keySet(),
           hasSize(record.preDistribCores.keySet().size()));
-      assertThat("multi pre-distrib command size", record.preDistribCommands, hasSize(100 + 10));
+      MatcherAssert.assertThat(
+          "multi pre-distrib command size", record.preDistribCommands, hasSize(100 + 10));
 
       assertEquals(
           "multi post-distrib must be same leaders",
@@ -372,7 +375,8 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
           "multi post-distrib cores don't match pre-distrib cores",
           record.preDistribCores.keySet(),
           record.postDistribCores.keySet());
-      assertThat("multi post-distrib command size", record.postDistribCommands, hasSize(100 + 10));
+      MatcherAssert.assertThat(
+          "multi post-distrib command size", record.postDistribCommands, hasSize(100 + 10));
     }
   }
 
@@ -391,65 +395,68 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
           assertUpdateWithRecording(prefPull(new UpdateRequest().add(sdoc("id", "hoss"))), client);
 
       // ...should start on (some) PULL replica, since we asked nicely
-      assertThat("add pre-distrib size", add.preDistribCores.keySet(), hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat("add pre-distrib size", add.preDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
           "add pre-distrib must be PULL",
           add.preDistribCores.keySet(),
           everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
-      assertThat("add pre-distrib size", add.preDistribRequests.keySet(), hasSize(1));
-      assertThat("add pre-distrib size", add.preDistribCommands, hasSize(1));
+      MatcherAssert.assertThat("add pre-distrib size", add.preDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("add pre-distrib size", add.preDistribCommands, hasSize(1));
 
       // ...then be routed to single leader for this id
-      assertThat("add post-distrib size", add.postDistribCores.keySet(), hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat("add post-distrib size", add.postDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
           "add post-distrib must be leader",
           add.postDistribCores.keySet(),
           everyItem(isIn(LEADER_CORE_NAMES)));
-      assertThat("add post-distrib size", add.postDistribRequests.keySet(), hasSize(1));
-      assertThat("add post-distrib size", add.postDistribCommands, hasSize(1));
+      MatcherAssert.assertThat(
+          "add post-distrib size", add.postDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("add post-distrib size", add.postDistribCommands, hasSize(1));
 
       // A DBI should also start on (some) PULL replica,  since we asked nicely.
       //
       // then it should be distributed to whatever leader our add doc (for the same id) was sent to
       final RecordingResults del =
           assertUpdateWithRecording(prefPull(new UpdateRequest().deleteById("hoss")), client);
-      assertThat("del pre-distrib size", del.preDistribCores.keySet(), hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat("del pre-distrib size", del.preDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
           "del pre-distrib must be PULL",
           del.preDistribCores.keySet(),
           everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
-      assertThat("del pre-distrib size", del.preDistribRequests.keySet(), hasSize(1));
-      assertThat("del pre-distrib size", del.preDistribCommands, hasSize(1));
+      MatcherAssert.assertThat("del pre-distrib size", del.preDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("del pre-distrib size", del.preDistribCommands, hasSize(1));
 
       assertEquals(
           "add and del should have same post-distrib leader",
           add.postDistribCores.keySet(),
           del.postDistribCores.keySet());
-      assertThat("del post-distrib size", del.postDistribRequests.keySet(), hasSize(1));
-      assertThat("del post-distrib size", del.postDistribCommands, hasSize(1));
+      MatcherAssert.assertThat(
+          "del post-distrib size", del.postDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("del post-distrib size", del.postDistribCommands, hasSize(1));
     }
 
     { // DBQ start on (some) PULL replica, since we asked nicely, then be routed to all leaders
       final RecordingResults record =
           assertUpdateWithRecording(prefPull(new UpdateRequest().deleteByQuery("*:*")), client);
 
-      assertThat("dbq pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat("dbq pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
           "dbq pre-distrib must be PULL",
           record.preDistribCores.keySet(),
           everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
-      assertThat("dbq pre-distrib size", record.preDistribRequests.keySet(), hasSize(1));
-      assertThat("dbq pre-distrib size", record.preDistribCommands, hasSize(1));
+      MatcherAssert.assertThat(
+          "dbq pre-distrib size", record.preDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("dbq pre-distrib size", record.preDistribCommands, hasSize(1));
 
       assertEquals(
           "dbq post-distrib must be all leaders",
           LEADER_CORE_NAMES,
           record.postDistribCores.keySet());
-      assertThat(
+      MatcherAssert.assertThat(
           "dbq post-distrib size",
           record.postDistribRequests.keySet(),
           hasSize(LEADER_CORE_NAMES.size()));
-      assertThat(
+      MatcherAssert.assertThat(
           "dbq post-distrib size", record.postDistribCommands, hasSize(LEADER_CORE_NAMES.size()));
     }
 
@@ -460,13 +467,16 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
       final RecordingResults record =
           assertUpdateWithRecording(prefPull(createMultiDirectUpdates(100, 10)), client);
 
-      assertThat("multi pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat(
+          "multi pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
           "multi pre-distrib must be PULL",
           record.preDistribCores.keySet(),
           everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
-      assertThat("multi pre-distrib req size", record.preDistribRequests.keySet(), hasSize(1));
-      assertThat("multi pre-distrib command size", record.preDistribCommands, hasSize(100 + 10));
+      MatcherAssert.assertThat(
+          "multi pre-distrib req size", record.preDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
+          "multi pre-distrib command size", record.preDistribCommands, hasSize(100 + 10));
 
       assertEquals(
           "multi post-distrib must be all leaders",
@@ -476,11 +486,12 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
       //
       // We make no asertion about number of post-distrb requests
       // (distrib proc may batch differently then what we send)
-      assertThat(
+      MatcherAssert.assertThat(
           "multi post-distrib cores",
           record.postDistribCores.keySet(),
           everyItem(isIn(LEADER_CORE_NAMES)));
-      assertThat("multi post-distrib command size", record.postDistribCommands, hasSize(100 + 10));
+      MatcherAssert.assertThat(
+          "multi post-distrib command size", record.postDistribCommands, hasSize(100 + 10));
     }
   }
 
@@ -497,22 +508,23 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
               client);
 
       // ...should start on (some) PULL replica, since we asked nicely
-      assertThat("add pre-distrib size", add.preDistribCores.keySet(), hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat("add pre-distrib size", add.preDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
           "add pre-distrib must be PULL",
           add.preDistribCores.keySet(),
           everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
-      assertThat("add pre-distrib size", add.preDistribRequests.keySet(), hasSize(1));
-      assertThat("add pre-distrib size", add.preDistribCommands, hasSize(1));
+      MatcherAssert.assertThat("add pre-distrib size", add.preDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("add pre-distrib size", add.preDistribCommands, hasSize(1));
 
       // ...then be routed to single leader for this id
-      assertThat("add post-distrib size", add.postDistribCores.keySet(), hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat("add post-distrib size", add.postDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
           "add post-distrib must be leader",
           add.postDistribCores.keySet(),
           everyItem(isIn(LEADER_CORE_NAMES)));
-      assertThat("add post-distrib size", add.postDistribRequests.keySet(), hasSize(1));
-      assertThat("add post-distrib size", add.postDistribCommands, hasSize(1));
+      MatcherAssert.assertThat(
+          "add post-distrib size", add.postDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("add post-distrib size", add.postDistribCommands, hasSize(1));
 
       // A DBI should also start on (some) PULL replica,  since we asked nicely.
       //
@@ -520,20 +532,21 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
       final RecordingResults del =
           assertUpdateWithRecording(
               prefPull(new UpdateRequest().deleteById("hoss")).setSendToLeaders(false), client);
-      assertThat("del pre-distrib size", del.preDistribCores.keySet(), hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat("del pre-distrib size", del.preDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
           "del pre-distrib must be PULL",
           del.preDistribCores.keySet(),
           everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
-      assertThat("del pre-distrib size", del.preDistribRequests.keySet(), hasSize(1));
-      assertThat("del pre-distrib size", del.preDistribCommands, hasSize(1));
+      MatcherAssert.assertThat("del pre-distrib size", del.preDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("del pre-distrib size", del.preDistribCommands, hasSize(1));
 
       assertEquals(
           "add and del should have same post-distrib leader",
           add.postDistribCores.keySet(),
           del.postDistribCores.keySet());
-      assertThat("del post-distrib size", del.postDistribRequests.keySet(), hasSize(1));
-      assertThat("del post-distrib size", del.postDistribCommands, hasSize(1));
+      MatcherAssert.assertThat(
+          "del post-distrib size", del.postDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("del post-distrib size", del.postDistribCommands, hasSize(1));
     }
 
     { // DBQ start on (some) PULL replica, since we asked nicely, then be routed to all leaders
@@ -541,23 +554,24 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
           assertUpdateWithRecording(
               prefPull(new UpdateRequest().deleteByQuery("*:*")).setSendToLeaders(false), client);
 
-      assertThat("dbq pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat("dbq pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
           "dbq pre-distrib must be PULL",
           record.preDistribCores.keySet(),
           everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
-      assertThat("dbq pre-distrib size", record.preDistribRequests.keySet(), hasSize(1));
-      assertThat("dbq pre-distrib size", record.preDistribCommands, hasSize(1));
+      MatcherAssert.assertThat(
+          "dbq pre-distrib size", record.preDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat("dbq pre-distrib size", record.preDistribCommands, hasSize(1));
 
       assertEquals(
           "dbq post-distrib must be all leaders",
           LEADER_CORE_NAMES,
           record.postDistribCores.keySet());
-      assertThat(
+      MatcherAssert.assertThat(
           "dbq post-distrib size",
           record.postDistribRequests.keySet(),
           hasSize(LEADER_CORE_NAMES.size()));
-      assertThat(
+      MatcherAssert.assertThat(
           "dbq post-distrib size", record.postDistribCommands, hasSize(LEADER_CORE_NAMES.size()));
     }
 
@@ -569,13 +583,16 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
           assertUpdateWithRecording(
               prefPull(createMultiDirectUpdates(100, 10)).setSendToLeaders(false), client);
 
-      assertThat("multi pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
-      assertThat(
+      MatcherAssert.assertThat(
+          "multi pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
           "multi pre-distrib must be PULL",
           record.preDistribCores.keySet(),
           everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
-      assertThat("multi pre-distrib req size", record.preDistribRequests.keySet(), hasSize(1));
-      assertThat("multi pre-distrib command size", record.preDistribCommands, hasSize(100 + 10));
+      MatcherAssert.assertThat(
+          "multi pre-distrib req size", record.preDistribRequests.keySet(), hasSize(1));
+      MatcherAssert.assertThat(
+          "multi pre-distrib command size", record.preDistribCommands, hasSize(100 + 10));
 
       assertEquals(
           "multi post-distrib must be all leaders",
@@ -585,11 +602,12 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
       //
       // We make no asertion about number of post-distrb requests
       // (distrib proc may batch differently then what we send)
-      assertThat(
+      MatcherAssert.assertThat(
           "multi post-distrib cores",
           record.postDistribCores.keySet(),
           everyItem(isIn(LEADER_CORE_NAMES)));
-      assertThat("multi post-distrib command size", record.postDistribCommands, hasSize(100 + 10));
+      MatcherAssert.assertThat(
+          "multi post-distrib command size", record.postDistribCommands, hasSize(100 + 10));
     }
   }
 


[solr] 12/12: Merge branch 'main' into jira/SOLR-6312

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit 029260f9888be3093797ba8adce0b3d091c718e4
Merge: 963f1ea3864 025c0305fa8
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Fri Jan 6 17:41:00 2023 -0700

    Merge branch 'main' into jira/SOLR-6312

 .github/workflows/bin-solr-test.yml                |     7 +
 NOTICE.txt                                         |     2 +-
 settings.gradle                                    |     2 +-
 solr/CHANGES.txt                                   |    52 +-
 solr/bin/solr                                      |   204 +-
 solr/bin/solr.cmd                                  |     5 +
 solr/bin/solr.in.cmd                               |     6 +-
 solr/bin/solr.in.sh                                |     4 +
 .../apache/solr/cloud/api/collections/Assign.java  |    37 +-
 ...istributedCollectionConfigSetCommandRunner.java |    12 +-
 .../solr/cloud/api/collections/SplitShardCmd.java  |   146 +-
 .../src/java/org/apache/solr/core/SolrCore.java    |     8 +-
 .../apache/solr/handler/MoreLikeThisHandler.java   |     2 +-
 .../solr/handler/admin/CollectionsHandler.java     |   194 +-
 .../solr/handler/admin/RebalanceLeaders.java       |    90 +-
 .../solr/handler/component/QueryComponent.java     |     2 +-
 .../handler/component/RealTimeGetComponent.java    |     4 +-
 .../java/org/apache/solr/search/QueryUtils.java    |     8 +-
 .../apache/solr/search/SolrDocumentFetcher.java    |     2 +-
 .../org/apache/solr/search/SolrIndexSearcher.java  |    20 +-
 .../org/apache/solr/search/neural/KnnQParser.java  |    33 +-
 .../apache/solr/update/StreamingSolrClients.java   |     3 +-
 .../org/apache/solr/update/UpdateShardHandler.java |    10 +-
 .../src/java/org/apache/solr/util/SolrCLI.java     |    27 +-
 .../collection1/conf/solrconfig-deeppaging.xml     |     2 +-
 .../apache/solr/cloud/ReindexCollectionTest.java   |     2 +-
 .../test/org/apache/solr/cloud/SplitShardTest.java |   194 +-
 .../solr/cloud/SystemCollectionCompatTest.java     |     2 +-
 .../solr/cloud/api/collections/ShardSplitTest.java |    13 +-
 .../solr/handler/admin/TestCollectionAPIs.java     |    15 +-
 .../org/apache/solr/search/LargeFieldTest.java     |    10 +-
 .../apache/solr/search/TestMainQueryCaching.java   |    50 +-
 .../apache/solr/search/neural/KnnQParserTest.java  |    57 +
 solr/example/films/README.md                       |    87 +-
 solr/example/films/films.csv                       |  2202 +--
 solr/example/films/films.json                      | 14202 ++++++++++++++++++-
 solr/example/films/films.xml                       | 11000 ++++++++++++++
 solr/example/films/vectors/README.md               |    47 +
 solr/example/films/vectors/create_dataset.py       |    68 +
 solr/example/films/vectors/create_model.py         |   110 +
 solr/example/films/vectors/films.py                |    92 +
 solr/licenses/annotations-4.1.1.4.jar.sha1         |     1 +
 solr/licenses/annotations-api-6.0.53.jar.sha1      |     1 +
 solr/licenses/grpc-LICENSE-ASL.txt                 |   610 +
 .../{grpc-context-NOTICE.txt => grpc-NOTICE.txt}   |     0
 solr/licenses/grpc-api-1.50.2.jar.sha1             |     1 +
 solr/licenses/grpc-context-1.48.0.jar.sha1         |     1 -
 solr/licenses/grpc-context-1.50.2.jar.sha1         |     1 +
 solr/licenses/grpc-context-LICENSE-ASL.txt         |   202 -
 solr/licenses/grpc-core-1.50.2.jar.sha1            |     1 +
 solr/licenses/grpc-netty-1.50.2.jar.sha1           |     1 +
 solr/licenses/grpc-protobuf-1.50.2.jar.sha1        |     1 +
 solr/licenses/grpc-protobuf-lite-1.50.2.jar.sha1   |     1 +
 solr/licenses/grpc-stub-1.50.2.jar.sha1            |     1 +
 solr/licenses/jaeger-core-1.8.1.jar.sha1           |     1 -
 solr/licenses/jaeger-core-NOTICE.txt               |     0
 solr/licenses/jaeger-thrift-1.8.1.jar.sha1         |     1 -
 solr/licenses/jaeger-thrift-LICENSE-ASL.txt        |   201 -
 solr/licenses/jaeger-thrift-NOTICE.txt             |     0
 solr/licenses/libthrift-0.15.0.jar.sha1            |     1 -
 solr/licenses/libthrift-NOTICE.txt                 |     5 -
 .../netty-codec-http2-4.1.82.Final.jar.sha1        |     1 +
 .../netty-codec-socks-4.1.82.Final.jar.sha1        |     1 +
 .../netty-handler-proxy-4.1.82.Final.jar.sha1      |     1 +
 solr/licenses/okhttp-4.10.0.jar.sha1               |     1 +
 solr/licenses/okhttp-4.9.3.jar.sha1                |     1 -
 solr/licenses/okio-jvm-2.8.0.jar.sha1              |     1 -
 solr/licenses/okio-jvm-3.0.0.jar.sha1              |     1 +
 ...CENSE-ASL.txt => opentelemetry-LICENSE-ASL.txt} |     2 +-
 solr/licenses/opentelemetry-NOTICE.txt             |     1 +
 solr/licenses/opentelemetry-api-1.21.0.jar.sha1    |     1 +
 .../opentelemetry-api-logs-1.21.0-alpha.jar.sha1   |     1 +
 .../licenses/opentelemetry-context-1.21.0.jar.sha1 |     1 +
 .../opentelemetry-exporter-common-1.21.0.jar.sha1  |     1 +
 .../opentelemetry-exporter-otlp-1.21.0.jar.sha1    |     1 +
 ...ntelemetry-exporter-otlp-common-1.21.0.jar.sha1 |     1 +
 ...elemetry-opentracing-shim-1.21.0-alpha.jar.sha1 |     1 +
 solr/licenses/opentelemetry-sdk-1.21.0.jar.sha1    |     1 +
 .../opentelemetry-sdk-common-1.21.0.jar.sha1       |     1 +
 ...k-extension-autoconfigure-1.21.0-alpha.jar.sha1 |     1 +
 ...sdk-extension-autoconfigure-spi-1.21.0.jar.sha1 |     1 +
 .../opentelemetry-sdk-logs-1.21.0-alpha.jar.sha1   |     1 +
 .../opentelemetry-sdk-metrics-1.21.0.jar.sha1      |     1 +
 .../opentelemetry-sdk-trace-1.21.0.jar.sha1        |     1 +
 .../opentelemetry-semconv-1.21.0-alpha.jar.sha1    |     1 +
 ...ft-LICENSE-ASL.txt => perfmark-LICENSE-ASL.txt} |     2 +-
 solr/licenses/perfmark-NOTICE.txt                  |    41 +
 solr/licenses/perfmark-api-0.25.0.jar.sha1         |     1 +
 solr/licenses/protobuf-java-3.21.12.jar.sha1       |     1 +
 solr/licenses/protobuf-java-3.21.8.jar.sha1        |     1 -
 solr/licenses/protobuf-java-util-3.21.12.jar.sha1  |     1 +
 solr/licenses/protobuf-java-util-3.21.4.jar.sha1   |     1 -
 .../org/apache/solr/handler/AnalyticsHandler.java  |     2 +-
 .../solr/security/hadoop/KerberosTestServices.java |     8 +-
 .../test/org/apache/hadoop/util/PlatformName.java  |    54 +
 solr/modules/jaegertracer-configurator/README.md   |    46 -
 .../modules/jaegertracer-configurator/build.gradle |    38 -
 .../solr/jaeger/JaegerTracerConfigurator.java      |    31 -
 .../apache/solr/jaeger/TestJaegerConfigurator.java |    67 -
 solr/modules/opentelemetry/README.md               |    26 +
 solr/modules/opentelemetry/build.gradle            |    53 +
 .../solr/opentelemetry/ClosableTracerShim.java     |    85 +
 .../solr/opentelemetry/OtelTracerConfigurator.java |   138 +
 .../apache/solr/opentelemetry}/package-info.java   |     4 +-
 .../src/java/overview.html                         |    11 +-
 .../src/test-files/log4j2.xml                      |     0
 .../test-files/solr/collection1/conf/schema.xml    |     0
 .../solr/collection1/conf/solrconfig.xml           |     0
 .../src/test-files/solr/solr.xml                   |     2 +-
 .../opentelemetry/OtelTracerConfiguratorTest.java  |   105 +
 solr/packaging/build.gradle                        |     7 +-
 solr/packaging/test/bats_helper.bash               |     2 +-
 ...lr.bats_disabled => test_placement_plugin.bats} |    21 +-
 ...art_solr.bats_disabled => test_start_solr.bats} |     2 +
 solr/solr-ref-guide/antora.template.yml            |     1 +
 solr/solr-ref-guide/antora.yml                     |     9 +-
 solr/solr-ref-guide/build.gradle                   |     1 +
 .../pages/package-manager-internals.adoc           |    46 +-
 .../pages/replica-placement-plugins.adoc           |     6 +-
 .../pages/cluster-node-management.adoc             |    13 +-
 .../pages/distributed-tracing.adoc                 |    72 +-
 .../deployment-guide/pages/shard-management.adoc   |    12 +
 .../modules/indexing-guide/pages/reindexing.adoc   |     2 +-
 .../indexing-guide/pages/schemaless-mode.adoc      |     8 +-
 .../query-guide/pages/dense-vector-search.adoc     |     7 +
 .../pages/major-changes-in-solr-10.adoc            |     7 +-
 .../pages/major-changes-in-solr-9.adoc             |     5 +
 .../client/solrj/impl/CloudHttp2SolrClient.java    |    42 +
 .../client/solrj/impl/CloudLegacySolrClient.java   |    16 +
 .../solr/client/solrj/impl/CloudSolrClient.java    |    20 +-
 .../impl/ConcurrentUpdateHttp2SolrClient.java      |    20 +
 .../solrj/impl/ConcurrentUpdateSolrClient.java     |    14 +-
 .../solrj/impl/DelegationTokenHttpSolrClient.java  |    18 +-
 .../solr/client/solrj/impl/Http2SolrClient.java    |    40 +-
 .../solr/client/solrj/impl/HttpSolrClient.java     |    40 +-
 .../solr/client/solrj/impl/LBHttp2SolrClient.java  |    83 +-
 .../solr/client/solrj/impl/LBHttpSolrClient.java   |    54 +-
 .../solr/client/solrj/impl/LBSolrClient.java       |    38 +-
 .../solr/client/solrj/impl/SolrClientBuilder.java  |     7 +-
 .../solrj/request/CollectionAdminRequest.java      |    11 +-
 .../apache/solr/client/solrj/util/Constants.java   |    22 +
 .../solr/common/params/CommonAdminParams.java      |     6 +
 .../solr/client/solrj/TestLBHttp2SolrClient.java   |    31 +-
 .../solr/client/solrj/TestLBHttpSolrClient.java    |    66 +-
 .../client/solrj/impl/BasicHttpSolrClientTest.java |    28 +-
 .../solrj/impl/CloudHttp2SolrClientTest.java       |    11 +-
 .../solrj/impl/CloudSolrClientCacheTest.java       |     2 +-
 .../client/solrj/impl/CloudSolrClientTest.java     |    11 +-
 .../impl/ConcurrentUpdateHttp2SolrClientTest.java  |     4 +-
 .../client/solrj/impl/Http2SolrClientTest.java     |    53 +-
 .../client/solrj/impl/LBHttp2SolrClientTest.java   |    53 +-
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   |    54 +-
 solr/webapp/web/js/angular/controllers/cloud.js    |     2 +-
 versions.lock                                      |    81 +-
 versions.props                                     |     5 +-
 155 files changed, 28983 insertions(+), 2877 deletions(-)

diff --cc solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudLegacySolrClient.java
index 1d817e109aa,983d959434c..eac2326a330
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudLegacySolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudLegacySolrClient.java
@@@ -266,12 -271,21 +271,23 @@@ public class CloudLegacySolrClient exte
        return this;
      }
  
+     /**
+      * Sets the cache ttl for DocCollection Objects cached.
+      *
+      * @param seconds ttl value in seconds
+      */
+     public Builder withCollectionCacheTtl(int seconds) {
+       assert seconds > 0;
+       this.timeToLiveSeconds = seconds;
+       return this;
+     }
+ 
      /**
 -     * Tells {@link Builder} that created clients should send updates only to shard leaders.
 +     * Tells {@link Builder} that created clients should be configured such that {@link
 +     * CloudSolrClient#isUpdatesToLeaders} returns <code>true</code>.
       *
 -     * <p>WARNING: This method currently has no effect. See SOLR-6312 for more information.
 +     * @see #sendUpdatesToAnyReplica
 +     * @see CloudSolrClient#isUpdatesToLeaders
       */
      public Builder sendUpdatesOnlyToShardLeaders() {
        shardLeadersOnly = true;
diff --cc solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java
index dc844ce627f,f994a216c6b..92409833464
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java
@@@ -302,9 -302,8 +302,9 @@@ public class CloudHttp2SolrClientTest e
  
      // Test Multi-Threaded routed updates for UpdateRequest
      try (CloudSolrClient threadedClient =
-         new CloudSolrClientBuilder(
+         new RandomizingCloudSolrClientBuilder(
                  Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
 +            .sendUpdatesOnlyToShardLeaders()
              .withParallelUpdates(true)
              .build()) {
        threadedClient.setDefaultCollection("routing_collection");
diff --cc solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
index 101b4787a05,61d2a60aaec..125c8a3580d
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
@@@ -303,9 -303,8 +303,9 @@@ public class CloudSolrClientTest extend
  
      // Test Multi-Threaded routed updates for UpdateRequest
      try (CloudSolrClient threadedClient =
-         new CloudSolrClientBuilder(
+         new RandomizingCloudSolrClientBuilder(
                  Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
 +            .sendUpdatesOnlyToShardLeaders()
              .withParallelUpdates(true)
              .build()) {
        threadedClient.setDefaultCollection("routing_collection");
diff --cc solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index 3c724202e1f,f05af27d6f7..6ae2eb76a2e
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@@ -2698,8 -2702,9 +2702,9 @@@ public abstract class SolrTestCaseJ4 ex
            .sendUpdatesOnlyToShardLeaders()
            .build();
      }
-     return new CloudSolrClientBuilder(Collections.singletonList(zkHost), Optional.empty())
+     return new RandomizingCloudSolrClientBuilder(
+             Collections.singletonList(zkHost), Optional.empty())
 -        .sendUpdatesToAllReplicasInShard()
 +        .sendUpdatesToAnyReplica()
          .build();
    }
  
@@@ -2720,8 -2727,9 +2727,9 @@@
            .withSocketTimeout(socketTimeoutMillis)
            .build();
      }
-     return new CloudSolrClientBuilder(Collections.singletonList(zkHost), Optional.empty())
+     return new RandomizingCloudSolrClientBuilder(
+             Collections.singletonList(zkHost), Optional.empty())
 -        .sendUpdatesToAllReplicasInShard()
 +        .sendUpdatesToAnyReplica()
          .withSocketTimeout(socketTimeoutMillis)
          .build();
    }
@@@ -2743,8 -2752,9 +2752,9 @@@
            .withSocketTimeout(socketTimeoutMillis)
            .build();
      }
-     return new CloudSolrClientBuilder(Collections.singletonList(zkHost), Optional.empty())
+     return new RandomizingCloudSolrClientBuilder(
+             Collections.singletonList(zkHost), Optional.empty())
 -        .sendUpdatesToAllReplicasInShard()
 +        .sendUpdatesToAnyReplica()
          .withConnectionTimeout(connectionTimeoutMillis)
          .withSocketTimeout(socketTimeoutMillis)
          .build();
@@@ -2763,9 -2774,10 +2774,10 @@@
            .sendUpdatesOnlyToShardLeaders()
            .build();
      }
-     return new CloudSolrClientBuilder(Collections.singletonList(zkHost), Optional.empty())
+     return new RandomizingCloudSolrClientBuilder(
+             Collections.singletonList(zkHost), Optional.empty())
          .withHttpClient(httpClient)
 -        .sendUpdatesToAllReplicasInShard()
 +        .sendUpdatesToAnyReplica()
          .build();
    }
  
@@@ -2788,9 -2801,10 +2801,10 @@@
            .withSocketTimeout(socketTimeoutMillis)
            .build();
      }
-     return new CloudSolrClientBuilder(Collections.singletonList(zkHost), Optional.empty())
+     return new RandomizingCloudSolrClientBuilder(
+             Collections.singletonList(zkHost), Optional.empty())
          .withHttpClient(httpClient)
 -        .sendUpdatesToAllReplicasInShard()
 +        .sendUpdatesToAnyReplica()
          .withConnectionTimeout(connectionTimeoutMillis)
          .withSocketTimeout(socketTimeoutMillis)
          .build();


[solr] 04/12: Fix tests that expect updates routed to leaders to explicitly configure sendUpdatesOnlyToShardLeaders() on their randomized client Builders

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit 52fe0227203cce92574bdb3163602854eefdd639
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Mon Dec 19 18:04:52 2022 -0700

    Fix tests that expect updates routed to leaders to explicitly configure sendUpdatesOnlyToShardLeaders() on their randomized client Builders
---
 .../test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java | 1 +
 .../src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java  | 1 +
 2 files changed, 2 insertions(+)

diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java
index f245c3bef04..dc844ce627f 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudHttp2SolrClientTest.java
@@ -304,6 +304,7 @@ public class CloudHttp2SolrClientTest extends SolrCloudTestCase {
     try (CloudSolrClient threadedClient =
         new CloudSolrClientBuilder(
                 Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            .sendUpdatesOnlyToShardLeaders()
             .withParallelUpdates(true)
             .build()) {
       threadedClient.setDefaultCollection("routing_collection");
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
index 6bd122f3402..101b4787a05 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
@@ -305,6 +305,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
     try (CloudSolrClient threadedClient =
         new CloudSolrClientBuilder(
                 Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            .sendUpdatesOnlyToShardLeaders()
             .withParallelUpdates(true)
             .build()) {
       threadedClient.setDefaultCollection("routing_collection");


[solr] 06/12: Beef up the assertions we can make by refactoring RecordingResults to track requests and commands (ot just core names)

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit 1ad2a161b785cee15db60be1052f6c7f9d78e199
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Tue Dec 20 11:25:06 2022 -0700

    Beef up the assertions we can make by refactoring RecordingResults to track requests and commands (ot just core names)
---
 .../impl/SendUpdatesToLeadersOverrideTest.java     | 147 ++++++++++++++++-----
 1 file changed, 114 insertions(+), 33 deletions(-)

diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
index af7d86a649b..1e72d6f16f1 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
@@ -23,9 +23,11 @@ import static org.hamcrest.Matchers.isIn;
 import static org.hamcrest.Matchers.not;
 
 import java.lang.invoke.MethodHandles;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
@@ -34,6 +36,8 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.update.UpdateCommand;
 import org.apache.solr.update.processor.TrackingUpdateProcessorFactory;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -122,14 +126,41 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
         .collect(Collectors.toUnmodifiableList());
   }
 
+  /** Convinience class for making assertions about the updates that were processed */
   private static class RecordingResults {
-    public final List<String> preDistribCoreNames;
-    public final List<String> postDistribCoreNames;
+    public final List<UpdateCommand> preDistribCommands;
+    public final List<UpdateCommand> postDistribCommands;
+
+    public final Map<SolrQueryRequest, List<UpdateCommand>> preDistribRequests;
+    public final Map<SolrQueryRequest, List<UpdateCommand>> postDistribRequests;
+
+    public final Map<String, List<SolrQueryRequest>> preDistribCores;
+    public final Map<String, List<SolrQueryRequest>> postDistribCores;
+
+    private static Map<SolrQueryRequest, List<UpdateCommand>> mapReqsToCommands(
+        final List<UpdateCommand> commands) {
+      return commands.stream().collect(Collectors.groupingBy(UpdateCommand::getReq));
+    }
+
+    private static Map<String, List<SolrQueryRequest>> mapCoresToReqs(
+        final Collection<SolrQueryRequest> reqs) {
+      return reqs.stream()
+          .collect(
+              Collectors.groupingBy(
+                  r -> r.getContext().get(TrackingUpdateProcessorFactory.REQUEST_NODE).toString()));
+    }
 
     public RecordingResults(
-        final List<String> preDistribCoreNames, final List<String> postDistribCoreNames) {
-      this.preDistribCoreNames = preDistribCoreNames;
-      this.postDistribCoreNames = postDistribCoreNames;
+        final List<UpdateCommand> preDistribCommands,
+        final List<UpdateCommand> postDistribCommands) {
+      this.preDistribCommands = preDistribCommands;
+      this.postDistribCommands = postDistribCommands;
+
+      this.preDistribRequests = mapReqsToCommands(preDistribCommands);
+      this.postDistribRequests = mapReqsToCommands(postDistribCommands);
+
+      this.preDistribCores = mapCoresToReqs(preDistribRequests.keySet());
+      this.postDistribCores = mapCoresToReqs(postDistribRequests.keySet());
     }
   }
 
@@ -148,13 +179,15 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
     assertEquals(0, req.process(client, COLLECTION_NAME).getStatus());
 
     final RecordingResults results =
-        new RecordingResults(stopRecording("pre-distrib"), stopRecording("post-distrib"));
+        new RecordingResults(
+            TrackingUpdateProcessorFactory.stopRecording("pre-distrib"),
+            TrackingUpdateProcessorFactory.stopRecording("post-distrib"));
 
     // post-distrib should never match any PULL replicas, regardless of request, if this fails
     // something is seriously wrong with our cluster
     assertThat(
         "post-distrib should never be PULL replica",
-        results.postDistribCoreNames,
+        results.postDistribCores.keySet(),
         everyItem(not(isIn(PULL_REPLICA_CORE_NAMES))));
 
     return results;
@@ -182,7 +215,6 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
     }
   }
 
-  /** nocommit: This test will fail until bug is fixed */
   public void testUpdatesWithShardsPrefPull() throws Exception {
     try (CloudSolrClient client =
         new CloudLegacySolrClient.Builder(
@@ -209,44 +241,71 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
           assertUpdateWithRecording(new UpdateRequest().add(sdoc("id", "hoss")), client);
 
       // single NRT leader is only core that should be involved at all
-      assertThat("add pre-distrib size", add.preDistribCoreNames, hasSize(1));
+      assertThat("add pre-distrib size", add.preDistribCores.keySet(), hasSize(1));
+      assertThat("add pre-distrib size", add.preDistribRequests.keySet(), hasSize(1));
+      assertThat("add pre-distrib size", add.preDistribCommands, hasSize(1));
       assertThat(
           "add pre-distrib must be leader",
-          add.preDistribCoreNames,
+          add.preDistribCores.keySet(),
           everyItem(isIn(LEADER_CORE_NAMES)));
       assertEquals(
-          "add pre and post should match", add.preDistribCoreNames, add.postDistribCoreNames);
+          "add pre and post should match",
+          add.preDistribCores.keySet(),
+          add.postDistribCores.keySet());
+      assertEquals(
+          "add pre and post should be exact same reqs",
+          add.preDistribRequests.keySet(),
+          add.postDistribRequests.keySet());
+      // NOTE: we can't assert the pre/post commands are the same, because they add versioning
 
       // whatever leader our add was routed to, a DBI for the same id should go to the same leader
       final RecordingResults del =
           assertUpdateWithRecording(new UpdateRequest().deleteById("hoss"), client);
       assertEquals(
-          "del pre and post should match", add.preDistribCoreNames, add.postDistribCoreNames);
+          "del pre and post should match",
+          del.preDistribCores.keySet(),
+          del.postDistribCores.keySet());
       assertEquals(
           "add and del should have been routed the same",
-          add.preDistribCoreNames,
-          del.preDistribCoreNames);
+          add.preDistribCores.keySet(),
+          del.preDistribCores.keySet());
+      assertThat("del pre-distrib size", del.preDistribRequests.keySet(), hasSize(1));
+      assertThat("del pre-distrib size", del.preDistribCommands, hasSize(1));
     }
 
     { // DBQ should start on some leader, and then distrib to both leaders
       final RecordingResults record =
           assertUpdateWithRecording(new UpdateRequest().deleteByQuery("*:*"), client);
 
-      assertThat("dbq pre-distrib size", record.preDistribCoreNames, hasSize(1));
+      assertThat("dbq pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
       assertThat(
           "dbq pre-distrib must be leader",
-          record.preDistribCoreNames,
+          record.preDistribCores.keySet(),
           everyItem(isIn(LEADER_CORE_NAMES)));
+      assertThat("dbq pre-distrib size", record.preDistribRequests.keySet(), hasSize(1));
+      assertThat("dbq pre-distrib size", record.preDistribCommands, hasSize(1));
 
       assertEquals(
           "dbq post-distrib must be all leaders",
           LEADER_CORE_NAMES,
-          new HashSet<>(record.postDistribCoreNames));
+          record.postDistribCores.keySet());
+      assertThat(
+          "dbq post-distrib size",
+          record.postDistribRequests.keySet(),
+          hasSize(LEADER_CORE_NAMES.size()));
+      assertThat(
+          "dbq post-distrib size", record.postDistribCommands, hasSize(LEADER_CORE_NAMES.size()));
     }
 
-    // nocommit: check an UpdateRequest with a mix of many doc adds DBIs, ...
-    // nocommit: confirm exactly 2 requests, one to each leader
+    { // When we have multiple direct updates for different shards, client will
+      // split them and merge the responses.
+      //
+      // But we should still only see at most one request per shard leader (pre and post)
 
+      // nocommit: check an UpdateRequest with a mix of many doc adds DBIs, ...
+      // nocommit: confirm no more then 2 pre-requests, one to each leader
+      // nocommit: confirm same pre and post requests
+    }
   }
 
   /**
@@ -264,53 +323,75 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
           assertUpdateWithRecording(prefPull(new UpdateRequest().add(sdoc("id", "hoss"))), client);
 
       // ...should start on (some) PULL replica, since we asked nicely
-      assertThat("add pre-distrib size", add.preDistribCoreNames, hasSize(1));
+      assertThat("add pre-distrib size", add.preDistribCores.keySet(), hasSize(1));
       assertThat(
           "add pre-distrib must be PULL",
-          add.preDistribCoreNames,
+          add.preDistribCores.keySet(),
           everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
+      assertThat("add pre-distrib size", add.preDistribRequests.keySet(), hasSize(1));
+      assertThat("add pre-distrib size", add.preDistribCommands, hasSize(1));
 
       // ...then be routed to single leader for this id
-      assertThat("add post-distrib size", add.postDistribCoreNames, hasSize(1));
+      assertThat("add post-distrib size", add.postDistribCores.keySet(), hasSize(1));
       assertThat(
           "add post-distrib must be leader",
-          add.postDistribCoreNames,
+          add.postDistribCores.keySet(),
           everyItem(isIn(LEADER_CORE_NAMES)));
+      assertThat("add post-distrib size", add.postDistribRequests.keySet(), hasSize(1));
+      assertThat("add post-distrib size", add.postDistribCommands, hasSize(1));
 
       // A DBI should also start on (some) PULL replica,  since we asked nicely.
       //
-      // then it should be distributed to whaever leader our add doc (for the same id) was sent to
+      // then it should be distributed to whatever leader our add doc (for the same id) was sent to
       final RecordingResults del =
           assertUpdateWithRecording(prefPull(new UpdateRequest().deleteById("hoss")), client);
-      assertThat("del pre-distrib size", del.preDistribCoreNames, hasSize(1));
+      assertThat("del pre-distrib size", del.preDistribCores.keySet(), hasSize(1));
       assertThat(
           "del pre-distrib must be PULL",
-          del.preDistribCoreNames,
+          del.preDistribCores.keySet(),
           everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
+      assertThat("del pre-distrib size", del.preDistribRequests.keySet(), hasSize(1));
+      assertThat("del pre-distrib size", del.preDistribCommands, hasSize(1));
+
       assertEquals(
           "add and del should have same post-distrib leader",
-          add.postDistribCoreNames,
-          del.postDistribCoreNames);
+          add.postDistribCores.keySet(),
+          del.postDistribCores.keySet());
+      assertThat("del post-distrib size", del.postDistribRequests.keySet(), hasSize(1));
+      assertThat("del post-distrib size", del.postDistribCommands, hasSize(1));
     }
 
     { // DBQ start on (some) PULL replica, since we asked nicely, then be routed to all leaders
       final RecordingResults record =
           assertUpdateWithRecording(prefPull(new UpdateRequest().deleteByQuery("*:*")), client);
 
-      assertThat("dbq pre-distrib size", record.preDistribCoreNames, hasSize(1));
+      assertThat("dbq pre-distrib size", record.preDistribCores.keySet(), hasSize(1));
       assertThat(
           "dbq pre-distrib must be PULL",
-          record.preDistribCoreNames,
+          record.preDistribCores.keySet(),
           everyItem(isIn(PULL_REPLICA_CORE_NAMES)));
+      assertThat("dbq pre-distrib size", record.preDistribRequests.keySet(), hasSize(1));
+      assertThat("dbq pre-distrib size", record.preDistribCommands, hasSize(1));
 
       assertEquals(
           "dbq post-distrib must be all leaders",
           LEADER_CORE_NAMES,
-          new HashSet<>(record.postDistribCoreNames));
+          record.postDistribCores.keySet());
+      assertThat(
+          "dbq post-distrib size",
+          record.postDistribRequests.keySet(),
+          hasSize(LEADER_CORE_NAMES.size()));
+      assertThat(
+          "dbq post-distrib size", record.postDistribCommands, hasSize(LEADER_CORE_NAMES.size()));
     }
 
-    // nocommit: check an UpdateRequest with a mix of many doc adds DBIs, ...
-    // nocommit: confirm we get exactly one request, no request splitting kicking in
+    { // When we have multiple direct updates for different shards, client will
+      // split them and merge the responses.
+      //
+      // But we should still only see at most one request per shard leader (pre and post)
 
+      // nocommit: check an UpdateRequest with a mix of many doc adds DBIs, ...
+      // nocommit: confirm we get exactly one pre request, no request splitting kicking in
+    }
   }
 }


[solr] 10/12: Add CloudHttp2SolrClient support and testing, fill in test class jdocs

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit a27567f8a92483b2fc20b5a29dbf7fc42d6744d3
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Fri Jan 6 15:46:45 2023 -0700

    Add CloudHttp2SolrClient support and testing, fill in test class jdocs
---
 .../client/solrj/impl/CloudHttp2SolrClient.java    | 32 ++++++++++++-
 .../impl/SendUpdatesToLeadersOverrideTest.java     | 53 +++++++++++++++++++---
 2 files changed, 78 insertions(+), 7 deletions(-)

diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
index 391a14b16fd..a41d8cbba64 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
@@ -136,7 +136,7 @@ public class CloudHttp2SolrClient extends CloudSolrClient {
     protected List<String> solrUrls = new ArrayList<>();
     protected String zkChroot;
     protected Http2SolrClient httpClient;
-    protected boolean shardLeadersOnly = true; // nocommit: use this, add setter
+    protected boolean shardLeadersOnly = true;
     protected boolean directUpdatesToLeadersOnly = false;
     protected boolean parallelUpdates = true;
     protected ClusterStateProvider stateProvider;
@@ -194,12 +194,39 @@ public class CloudHttp2SolrClient extends CloudSolrClient {
       if (zkChroot.isPresent()) this.zkChroot = zkChroot.get();
     }
 
+    /**
+     * Tells {@link Builder} that created clients should be configured such that {@link
+     * CloudSolrClient#isUpdatesToLeaders} returns <code>true</code>.
+     *
+     * @see #sendUpdatesToAnyReplica
+     * @see CloudSolrClient#isUpdatesToLeaders
+     */
+    public Builder sendUpdatesOnlyToShardLeaders() {
+      shardLeadersOnly = true;
+      return this;
+    }
+
+    /**
+     * Tells {@link Builder} that created clients should be configured such that {@link
+     * CloudSolrClient#isUpdatesToLeaders} returns <code>false</code>.
+     *
+     * @see #sendUpdatesOnlyToShardLeaders
+     * @see CloudSolrClient#isUpdatesToLeaders
+     */
+    public Builder sendUpdatesToAnyReplica() {
+      shardLeadersOnly = false;
+      return this;
+    }
+
     /**
      * Tells {@link CloudHttp2SolrClient.Builder} that created clients should send direct updates to
      * shard leaders only.
      *
      * <p>UpdateRequests whose leaders cannot be found will "fail fast" on the client side with a
      * {@link SolrException}
+     *
+     * @see #sendDirectUpdatesToAnyShardReplica
+     * @see CloudSolrClient#isDirectUpdatesToLeadersOnly
      */
     public Builder sendDirectUpdatesToShardLeadersOnly() {
       directUpdatesToLeadersOnly = true;
@@ -212,6 +239,9 @@ public class CloudHttp2SolrClient extends CloudSolrClient {
      *
      * <p>Shard leaders are still preferred, but the created clients will fallback to using other
      * replicas if a leader cannot be found.
+     *
+     * @see #sendDirectUpdatesToShardLeadersOnly
+     * @see CloudSolrClient#isDirectUpdatesToLeadersOnly
      */
     public Builder sendDirectUpdatesToAnyShardReplica() {
       directUpdatesToLeadersOnly = false;
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
index c185e1e4d82..9a485a483b3 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
@@ -34,6 +34,7 @@ import java.util.stream.Collectors;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.IsUpdateRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.cloud.Replica;
@@ -46,9 +47,15 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Test the behavior of {@link CloudSolrClient#isUpdatesToLeaders}
+ * Test the behavior of {@link CloudSolrClient#isUpdatesToLeaders} and {@link
+ * IsUpdateRequest#isSendToLeaders}.
  *
- * <p>nocommit: more explanation of how we test this
+ * <p>This class uses {@link TrackingUpdateProcessorFactory} instances (configured both before, and
+ * after the <code>distrib</code> processor) to inspect which replicas recieve various {@link
+ * UpdateRequest}s from variously configured {@link CloudSolrClient}s. In some requests, <code>
+ * shards.preference=replica.type:PULL</code> is specified to confirm that typical routing
+ * prefrences are respected (when the effective value of <code>isSendToLeaders</code> is <code>false
+ * </code>)
  */
 public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
 
@@ -204,10 +211,22 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
     return req;
   }
 
-  // nocommit: - test CloudHttp2SolrClient as well
+  public void testBuilderImplicitBehavior() throws Exception {
+    try (CloudSolrClient client =
+        new CloudLegacySolrClient.Builder(
+                Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            .build()) {
+      assertTrue(client.isUpdatesToLeaders());
+    }
+    try (CloudSolrClient client =
+        new CloudHttp2SolrClient.Builder(
+                Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            .build()) {
+      assertTrue(client.isUpdatesToLeaders());
+    }
+  }
 
-  // basic sanity check of expected default behavior
-  public void testClientThatDefaultsToLeaders() throws Exception {
+  public void testLegacyClientThatDefaultsToLeaders() throws Exception {
     try (CloudSolrClient client =
         new CloudLegacySolrClient.Builder(
                 Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
@@ -218,7 +237,7 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
     }
   }
 
-  public void testClientThatDoesNotDefaultToLeaders() throws Exception {
+  public void testLegacyClientThatDoesNotDefaultToLeaders() throws Exception {
     try (CloudSolrClient client =
         new CloudLegacySolrClient.Builder(
                 Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
@@ -229,6 +248,28 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
     }
   }
 
+  public void testHttp2ClientThatDefaultsToLeaders() throws Exception {
+    try (CloudSolrClient client =
+        new CloudHttp2SolrClient.Builder(
+                Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            .sendUpdatesOnlyToShardLeaders()
+            .build()) {
+      checkUpdatesDefaultToLeaders(client);
+      checkUpdatesWithSendToLeadersFalse(client);
+    }
+  }
+
+  public void testHttp2ClientThatDoesNotDefaultToLeaders() throws Exception {
+    try (CloudSolrClient client =
+        new CloudHttp2SolrClient.Builder(
+                Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
+            .sendUpdatesToAnyReplica()
+            .build()) {
+      checkUpdatesWithShardsPrefPull(client);
+      checkUpdatesWithSendToLeadersFalse(client);
+    }
+  }
+
   /**
    * Given a SolrClient, sends various updates and asserts expecations regarding default behavior:
    * that these requests will be initially sent to shard leaders, and "routed" requests will be sent


[solr] 09/12: Fix Builder method name and update javadocs

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit e9c4f80659a110e340db673dd2f2a3ede77b8112
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Fri Jan 6 12:21:15 2023 -0700

    Fix Builder method name and update javadocs
---
 .../client/solrj/impl/CloudHttp2SolrClient.java    |  2 +-
 .../client/solrj/impl/CloudLegacySolrClient.java   | 40 +++++++++++++++++++---
 .../impl/SendUpdatesToLeadersOverrideTest.java     |  4 +--
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   | 10 +++---
 4 files changed, 42 insertions(+), 14 deletions(-)

diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
index 49200f4ce2f..391a14b16fd 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudHttp2SolrClient.java
@@ -136,7 +136,7 @@ public class CloudHttp2SolrClient extends CloudSolrClient {
     protected List<String> solrUrls = new ArrayList<>();
     protected String zkChroot;
     protected Http2SolrClient httpClient;
-    protected boolean shardLeadersOnly = true;
+    protected boolean shardLeadersOnly = true; // nocommit: use this, add setter
     protected boolean directUpdatesToLeadersOnly = false;
     protected boolean parallelUpdates = true;
     protected ClusterStateProvider stateProvider;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudLegacySolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudLegacySolrClient.java
index 58da3d71d3f..1d817e109aa 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudLegacySolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudLegacySolrClient.java
@@ -267,9 +267,11 @@ public class CloudLegacySolrClient extends CloudSolrClient {
     }
 
     /**
-     * Tells {@link Builder} that created clients should send updates only to shard leaders.
+     * Tells {@link Builder} that created clients should be configured such that {@link
+     * CloudSolrClient#isUpdatesToLeaders} returns <code>true</code>.
      *
-     * <p>WARNING: This method currently has no effect. See SOLR-6312 for more information.
+     * @see #sendUpdatesToAnyReplica
+     * @see CloudSolrClient#isUpdatesToLeaders
      */
     public Builder sendUpdatesOnlyToShardLeaders() {
       shardLeadersOnly = true;
@@ -277,20 +279,45 @@ public class CloudLegacySolrClient extends CloudSolrClient {
     }
 
     /**
-     * Tells {@link Builder} that created clients should send updates to all replicas for a shard.
+     * Tells {@link Builder} that created clients should be configured such that {@link
+     * CloudSolrClient#isUpdatesToLeaders} returns <code>false</code>.
      *
-     * <p>WARNING: This method currently has no effect. See SOLR-6312 for more information.
+     * @see #sendUpdatesOnlyToShardLeaders
+     * @see CloudSolrClient#isUpdatesToLeaders
      */
-    public Builder sendUpdatesToAllReplicasInShard() {
+    public Builder sendUpdatesToAnyReplica() {
       shardLeadersOnly = false;
       return this;
     }
 
+    /**
+     * This method has no effect.
+     *
+     * <p>In older versions of Solr, this method was an incorrectly named equivilent to {@link
+     * #sendUpdatesToAnyReplica}, which had no effect because that setting was ignored in the
+     * created clients. When the underlying {@link CloudSolrClient} behavior was fixed, this method
+     * was modified to be an explicit No-Op, since the implied behavior of sending updates to
+     * <em>all</em> replicas has never been supported, and was never intended to be supported.
+     *
+     * @see #sendUpdatesOnlyToShardLeaders
+     * @see #sendUpdatesToAnyReplica
+     * @see CloudSolrClient#isUpdatesToLeaders
+     * @see <a href="https://issues.apache.org/jira/browse/SOLR-6312">SOLR-6312</a>
+     * @deprecated Never supported
+     */
+    @Deprecated
+    public Builder sendUpdatesToAllReplicasInShard() {
+      return this;
+    }
+
     /**
      * Tells {@link Builder} that created clients should send direct updates to shard leaders only.
      *
      * <p>UpdateRequests whose leaders cannot be found will "fail fast" on the client side with a
      * {@link SolrException}
+     *
+     * @see #sendDirectUpdatesToAnyShardReplica
+     * @see CloudSolrClient#isDirectUpdatesToLeadersOnly
      */
     public Builder sendDirectUpdatesToShardLeadersOnly() {
       directUpdatesToLeadersOnly = true;
@@ -303,6 +330,9 @@ public class CloudLegacySolrClient extends CloudSolrClient {
      *
      * <p>Shard leaders are still preferred, but the created clients will fallback to using other
      * replicas if a leader cannot be found.
+     *
+     * @see #sendDirectUpdatesToShardLeadersOnly
+     * @see CloudSolrClient#isDirectUpdatesToLeadersOnly
      */
     public Builder sendDirectUpdatesToAnyShardReplica() {
       directUpdatesToLeadersOnly = false;
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
index 044c24f339d..c185e1e4d82 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/SendUpdatesToLeadersOverrideTest.java
@@ -222,9 +222,7 @@ public class SendUpdatesToLeadersOverrideTest extends SolrCloudTestCase {
     try (CloudSolrClient client =
         new CloudLegacySolrClient.Builder(
                 Collections.singletonList(cluster.getZkServer().getZkAddress()), Optional.empty())
-            // nocommit: builder name should never have been 'sendUpdatesToAllReplicasInShard'
-            // nocommit: - should have been 'sendUpdatesToAnyReplicas'
-            .sendUpdatesToAllReplicasInShard()
+            .sendUpdatesToAnyReplica()
             .build()) {
       checkUpdatesWithShardsPrefPull(client);
       checkUpdatesWithSendToLeadersFalse(client);
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 7caa396578b..3c724202e1f 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -2699,7 +2699,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
           .build();
     }
     return new CloudSolrClientBuilder(Collections.singletonList(zkHost), Optional.empty())
-        .sendUpdatesToAllReplicasInShard()
+        .sendUpdatesToAnyReplica()
         .build();
   }
 
@@ -2721,7 +2721,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
           .build();
     }
     return new CloudSolrClientBuilder(Collections.singletonList(zkHost), Optional.empty())
-        .sendUpdatesToAllReplicasInShard()
+        .sendUpdatesToAnyReplica()
         .withSocketTimeout(socketTimeoutMillis)
         .build();
   }
@@ -2744,7 +2744,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
           .build();
     }
     return new CloudSolrClientBuilder(Collections.singletonList(zkHost), Optional.empty())
-        .sendUpdatesToAllReplicasInShard()
+        .sendUpdatesToAnyReplica()
         .withConnectionTimeout(connectionTimeoutMillis)
         .withSocketTimeout(socketTimeoutMillis)
         .build();
@@ -2765,7 +2765,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
     }
     return new CloudSolrClientBuilder(Collections.singletonList(zkHost), Optional.empty())
         .withHttpClient(httpClient)
-        .sendUpdatesToAllReplicasInShard()
+        .sendUpdatesToAnyReplica()
         .build();
   }
 
@@ -2790,7 +2790,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
     }
     return new CloudSolrClientBuilder(Collections.singletonList(zkHost), Optional.empty())
         .withHttpClient(httpClient)
-        .sendUpdatesToAllReplicasInShard()
+        .sendUpdatesToAnyReplica()
         .withConnectionTimeout(connectionTimeoutMillis)
         .withSocketTimeout(socketTimeoutMillis)
         .build();


[solr] 03/12: Fix root cause of bug

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit cb52dd264e51d263b30f95802237521e088c7ebd
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Fri Dec 16 16:09:08 2022 -0700

    Fix root cause of bug
---
 .../java/org/apache/solr/client/solrj/impl/CloudSolrClient.java   | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)

diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index b435ef72233..56a9c4e14c4 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -1046,7 +1046,12 @@ public abstract class CloudSolrClient extends SolrClient {
     boolean isUpdate = false;
 
     if (request instanceof IsUpdateRequest) {
-      if (request instanceof UpdateRequest) {
+      sendToLeaders =
+          // nocommit: also check request.isSendToLeaders() (still to be added)
+          this.isUpdatesToLeaders();
+
+      // Check if we can do a "directUpdate" ...
+      if (sendToLeaders && request instanceof UpdateRequest) {
         isUpdate = true;
         if (inputCollections.size() > 1) {
           throw new SolrException(
@@ -1064,7 +1069,6 @@ public abstract class CloudSolrClient extends SolrClient {
           return response;
         }
       }
-      sendToLeaders = true;
     }
 
     SolrParams reqParams = request.getParams();


[solr] 01/12: refactor TrackingUpdateProcessorFactory (and RecordingUpdateProcessorFactory) into test-framework so they can be re-used in solrj tests

Posted by ho...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

hossman pushed a commit to branch jira/SOLR-6312
in repository https://gitbox.apache.org/repos/asf/solr.git

commit b4edbd3d172ba689f5aef89736d4c7be4e903357
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Thu Dec 15 12:40:24 2022 -0700

    refactor TrackingUpdateProcessorFactory (and RecordingUpdateProcessorFactory) into test-framework so they can be re-used in solrj tests
---
 .../org/apache/solr/update/processor/RecordingUpdateProcessorFactory.java | 0
 .../org/apache/solr/update/processor/TrackingUpdateProcessorFactory.java  | 0
 2 files changed, 0 insertions(+), 0 deletions(-)

diff --git a/solr/core/src/test/org/apache/solr/update/processor/RecordingUpdateProcessorFactory.java b/solr/test-framework/src/java/org/apache/solr/update/processor/RecordingUpdateProcessorFactory.java
similarity index 100%
rename from solr/core/src/test/org/apache/solr/update/processor/RecordingUpdateProcessorFactory.java
rename to solr/test-framework/src/java/org/apache/solr/update/processor/RecordingUpdateProcessorFactory.java
diff --git a/solr/core/src/test/org/apache/solr/update/processor/TrackingUpdateProcessorFactory.java b/solr/test-framework/src/java/org/apache/solr/update/processor/TrackingUpdateProcessorFactory.java
similarity index 100%
rename from solr/core/src/test/org/apache/solr/update/processor/TrackingUpdateProcessorFactory.java
rename to solr/test-framework/src/java/org/apache/solr/update/processor/TrackingUpdateProcessorFactory.java