You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ge...@apache.org on 2018/10/07 14:08:58 UTC

lucene-solr:master: SOLR-12803: Ensure CUSC routes docs to right cores

Repository: lucene-solr
Updated Branches:
  refs/heads/master e2b8beccb -> 367bdf7f7


SOLR-12803: Ensure CUSC routes docs to right cores

ConcurrentUpdateSolrClient can batch together many documents when making
an indexing request to Solr.  When adding an update request to the
current batch being made, it checks that the query-parameters of the
docs being added match those already in the batch.  But prior to this
commit it never checked that the collections/cores were the same.

This could result in documents being sent to the wrong collection if the
same client is used to index documents to two different
cores/collections simultaneously.

This commit addresses this problem, ensuring that documents aren't added
to a batch directed at a different core/collection.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/367bdf7f
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/367bdf7f
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/367bdf7f

Branch: refs/heads/master
Commit: 367bdf7f749a4386071f4444e2d6a09591f38daf
Parents: e2b8bec
Author: Jason Gerlowski <ge...@apache.org>
Authored: Sun Oct 7 09:51:43 2018 -0400
Committer: Jason Gerlowski <ge...@apache.org>
Committed: Sun Oct 7 10:08:50 2018 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  2 +
 .../solrj/impl/ConcurrentUpdateSolrClient.java  |  6 +-
 .../org/apache/solr/common/StringUtils.java     | 10 +++
 ...rentUpdateSolrClientMultiCollectionTest.java | 94 ++++++++++++++++++++
 4 files changed, 110 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/367bdf7f/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4bd8158..4fea2ff 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -148,6 +148,8 @@ Other Changes
 Bug Fixes
 ----------------------
 
+* SOLR-12803: Ensure ConcurrentUpdateSolrClient sends documents to correct collection (Jason Gerlowski)
+
 * SOLR-11836: FacetStream works with bucketSizeLimit of -1 which will fetch all the buckets.
   (Alfonso Muñoz-Pomer Fuentes, Amrit Sarkar via Varun Thacker)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/367bdf7f/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
index 5845e7f..26af757 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
@@ -47,6 +47,7 @@ import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.StringUtils;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
@@ -243,6 +244,7 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
           final boolean isXml = ClientUtils.TEXT_XML.equals(contentType);
 
           final ModifiableSolrParams origParams = new ModifiableSolrParams(update.getRequest().getParams());
+          final String origTargetCollection = update.getCollection();
 
           EntityTemplate template = new EntityTemplate(new ContentProducer() {
             
@@ -256,8 +258,8 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
               while (upd != null) {
                 UpdateRequest req = upd.getRequest();
                 SolrParams currentParams = new ModifiableSolrParams(req.getParams());
-                if (!origParams.toNamedList().equals(currentParams.toNamedList())) {
-                  queue.add(upd); // params are different, push back to queue
+                if (!origParams.toNamedList().equals(currentParams.toNamedList()) || !StringUtils.equals(origTargetCollection, upd.getCollection())) {
+                  queue.add(upd); // Request has different params or destination core/collection, return to queue
                   break;
                 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/367bdf7f/solr/solrj/src/java/org/apache/solr/common/StringUtils.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/StringUtils.java b/solr/solrj/src/java/org/apache/solr/common/StringUtils.java
index 9b8e042..58382a7 100644
--- a/solr/solrj/src/java/org/apache/solr/common/StringUtils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/StringUtils.java
@@ -22,4 +22,14 @@ public class StringUtils {
     return (s == null) || s.isEmpty();
   }
 
+  /**
+   * A "null-safe" equals method.  Returns true if the two provided references are both null, or if they are string-equal.
+   */
+  public static boolean equals(String first, String second) {
+    if (first == null) {
+      return second == null;
+    }
+    return first.equals(second);
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/367bdf7f/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientMultiCollectionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientMultiCollectionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientMultiCollectionTest.java
new file mode 100644
index 0000000..9823740
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientMultiCollectionTest.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.impl;
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * {@link ConcurrentUpdateSolrClient} reuses the same HTTP connection to send multiple requests.  These tests ensure
+ * that this connection-reuse never results in documents being sent to the wrong collection.  See SOLR-12803
+ */
+public class ConcurrentUpdateSolrClientMultiCollectionTest extends SolrCloudTestCase {
+
+  private static final String COLLECTION_ONE_NAME = "collection1";
+  private static final String COLLECTION_TWO_NAME = "collection2";
+
+  private String solrUrl;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig("conf", new File(ExternalPaths.TECHPRODUCTS_CONFIGSET).toPath())
+        .configure();
+  }
+
+  @Before
+  public void createCollections() throws Exception {
+    solrUrl = cluster.getJettySolrRunner(0).getBaseUrl().toString();
+
+    CollectionAdminRequest.createCollection(COLLECTION_ONE_NAME, "conf", 1, 1).process(cluster.getSolrClient());
+    CollectionAdminRequest.createCollection(COLLECTION_TWO_NAME, "conf", 1, 1).process(cluster.getSolrClient());
+  }
+
+  @After
+  public void deleteCollections() throws Exception {
+    cluster.deleteAllCollections();
+  }
+
+  @Test
+  public void testEnsureDocumentsSentToCorrectCollection() throws Exception {
+    int numTotalDocs = 1000;
+    int numExpectedPerCollection = numTotalDocs / 2;
+    try (SolrClient client = new ConcurrentUpdateSolrClient.Builder(solrUrl)
+        .withQueueSize(numTotalDocs).build()) {
+      splitDocumentsAcrossCollections(client, numTotalDocs);
+
+      assertEquals(numExpectedPerCollection, client.query(COLLECTION_ONE_NAME, new SolrQuery("*:*")).getResults().getNumFound());
+      assertEquals(numExpectedPerCollection, client.query(COLLECTION_TWO_NAME, new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+  }
+
+  private void splitDocumentsAcrossCollections(SolrClient client, int numTotalDocs) throws IOException, SolrServerException {
+    for (int docNum = 0; docNum < numTotalDocs; docNum++) {
+      final SolrInputDocument doc = new SolrInputDocument();
+      doc.setField("id", "value" + docNum);
+
+      if (docNum %2 == 0) {
+        client.add(COLLECTION_ONE_NAME, doc);
+      } else {
+        client.add(COLLECTION_TWO_NAME, doc);
+      }
+    }
+
+    client.commit(COLLECTION_ONE_NAME);
+    client.commit(COLLECTION_TWO_NAME);
+  }
+}