You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by er...@apache.org on 2020/02/21 15:57:23 UTC

[lucene-solr] branch master updated: SOLR-11035: (at least) 2 distinct failures possible when clients attempt searches during SolrCore reload

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

erick pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 292bed0  SOLR-11035: (at least) 2 distinct failures possible when clients attempt searches during SolrCore reload
292bed0 is described below

commit 292bed0c429644226a8218c138845163b261a5c8
Author: Erick Erickson <Er...@gmail.com>
AuthorDate: Fri Feb 21 10:57:16 2020 -0500

    SOLR-11035: (at least) 2 distinct failures possible when clients attempt searches during SolrCore reload
---
 solr/CHANGES.txt                                   |   3 +
 .../apache/solr/cloud/DocValuesNotIndexedTest.java |   5 -
 .../apache/solr/cloud/ReindexCollectionTest.java   |  14 ---
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   | 123 ---------------------
 4 files changed, 3 insertions(+), 142 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b7adc7c..11bcc42 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -197,6 +197,9 @@ Other Changes
 
 * SOLR-14263: Update jvm-settings.adoc (Erick Erickson)
 
+* SOLR-11035: (at least) 2 distinct failures possible when clients attempt searches
+  during SolrCore reload. Removed Bandaid kludge. (Erick Erickson)
+
 ==================  8.4.1 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
diff --git a/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java b/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
index a1c5961..5fa604e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
@@ -179,7 +179,6 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
     CloudSolrClient client = cluster.getSolrClient();
     client.deleteByQuery("*:*");
     client.commit();
-    Solr11035BandAid(client, COLLECTION, "id", 0, "*:*", "DocValuesNotINdexedTest.clean");
     resetFields(fieldsToTestSingle);
     resetFields(fieldsToTestMulti);
     resetFields(fieldsToTestGroupSortFirst);
@@ -255,7 +254,6 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
     new UpdateRequest()
         .add(docs)
         .commit(client, COLLECTION);
-    Solr11035BandAid(client, COLLECTION, "id", 4, "*:*", "DocValuesNotINdexedTest.testGroupSorting");
 
     checkSortOrder(client, fieldsToTestGroupSortFirst, "asc", new String[]{"4", "2", "1", "3"}, new String[]{"4", "1", "2", "3"});
     checkSortOrder(client, fieldsToTestGroupSortFirst, "desc", new String[]{"3", "1", "2", "4"}, new String[]{"2", "3", "1", "4"});
@@ -298,7 +296,6 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
         .add(docs)
         .commit(client, COLLECTION);
 
-    Solr11035BandAid(client, COLLECTION, "id", 4, "*:*", "DocValuesNotINdexedTest.testGroupingDocAbsent");
     // when grouping on any of these DV-only (not indexed) fields we expect exactly 4 groups except for Boolean.
     for (FieldProps prop : fieldsToTestGroupSortFirst) {
       // Special handling until SOLR-9802 is fixed
@@ -362,8 +359,6 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
         .add(docs)
         .commit(client, COLLECTION);
 
-    Solr11035BandAid(client, COLLECTION,"id", 59, "*:*", "DocValuesNotINdexedTest.doGroupingDvOnly");
-
     // OK, we should have one group with 10 entries for null, a group with 1 entry and 7 groups with 7
     for (FieldProps prop : fieldProps) {
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/ReindexCollectionTest.java b/solr/core/src/test/org/apache/solr/cloud/ReindexCollectionTest.java
index f03e4f8..a2a6de8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ReindexCollectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ReindexCollectionTest.java
@@ -26,7 +26,6 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 
-import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.cloud.DistribStateManager;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -142,9 +141,6 @@ public class ReindexCollectionTest extends SolrCloudTestCase {
       return ReindexCollectionCmd.State.FINISHED == state;
     });
 
-    SolrTestCaseJ4.Solr11035BandAid(solrClient, targetCollection, "id", NUM_DOCS, "*:*",
-        "ReindexCollectionTest.testBasicReindexing", false);
-
     // verify the target docs exist
     QueryResponse queryResponse = solrClient.query(targetCollection, params(CommonParams.Q, "*:*"));
     assertEquals("copied num docs", NUM_DOCS, queryResponse.getResults().getNumFound());
@@ -194,8 +190,6 @@ public class ReindexCollectionTest extends SolrCloudTestCase {
       return ReindexCollectionCmd.State.FINISHED == state;
     });
     solrClient.getZkStateReader().aliasesManager.update();
-    SolrTestCaseJ4.Solr11035BandAid(solrClient, targetCollection, "id", NUM_DOCS, "*:*",
-        "ReindexCollectionTest.testSameTargetReindex_" + sourceRemove, false);
     // verify the target docs exist
     QueryResponse rsp = solrClient.query(targetCollection, params(CommonParams.Q, "*:*"));
     assertEquals("copied num docs", NUM_DOCS, rsp.getResults().getNumFound());
@@ -228,8 +222,6 @@ public class ReindexCollectionTest extends SolrCloudTestCase {
       ReindexCollectionCmd.State state = ReindexCollectionCmd.State.get(coll.getStr(ReindexCollectionCmd.REINDEXING_STATE));
       return ReindexCollectionCmd.State.FINISHED == state;
     });
-    SolrTestCaseJ4.Solr11035BandAid(solrClient, targetCollection, "id", NUM_DOCS, "*:*",
-        "ReindexCollectionTest.testLossyScherma", false);
     // verify the target docs exist
     QueryResponse rsp = solrClient.query(targetCollection, params(CommonParams.Q, "*:*"));
     assertEquals("copied num docs", NUM_DOCS, rsp.getResults().getNumFound());
@@ -265,9 +257,6 @@ public class ReindexCollectionTest extends SolrCloudTestCase {
       ReindexCollectionCmd.State state = ReindexCollectionCmd.State.get(coll.getStr(ReindexCollectionCmd.REINDEXING_STATE));
       return ReindexCollectionCmd.State.FINISHED == state;
     });
-    SolrTestCaseJ4.Solr11035BandAid(solrClient, targetCollection, "id", 11, "*:*",
-        "ReindexCollectionTest.testReshapeReindexTarget", false);
-
     // verify the target docs exist
     QueryResponse rsp = solrClient.query(targetCollection, params(CommonParams.Q, "*:*"));
 
@@ -402,9 +391,6 @@ public class ReindexCollectionTest extends SolrCloudTestCase {
     }
     solrClient.add(collection, docs);
     solrClient.commit(collection);
-    SolrTestCaseJ4.Solr11035BandAid(solrClient, collection, "id", NUM_DOCS, "*:*",
-        "ReindexCollectionTest.indexDocs", false);
-
     // verify the docs exist
     QueryResponse rsp = solrClient.query(collection, params(CommonParams.Q, "*:*"));
 
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 28e8bdb..58d44a7 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -84,9 +84,6 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 import org.apache.lucene.util.QuickPatchThreadsFilter;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.client.solrj.ResponseParser;
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrQuery;
-import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettyConfig;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.CloudHttp2SolrClient;
@@ -98,8 +95,6 @@ import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder;
 import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
-import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.SolrResponseBase;
 import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.cloud.IpTables;
@@ -120,7 +115,6 @@ import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.SolrjNamedThreadFactory;
 import org.apache.solr.common.util.SuppressForbidden;
-import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.common.util.XML;
 import org.apache.solr.core.CoreContainer;
@@ -152,7 +146,6 @@ import org.apache.solr.util.SSLTestConfig;
 import org.apache.solr.util.StartupLoggingUtils;
 import org.apache.solr.util.TestHarness;
 import org.apache.solr.util.TestInjection;
-import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -3075,120 +3068,4 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
   protected static final Map<Class,String> RANDOMIZED_NUMERIC_FIELDTYPES
     = Collections.unmodifiableMap(private_RANDOMIZED_NUMERIC_FIELDTYPES);
 
-
-  /**
-   * See SOLR-11035. There are various "impossible" failures, I can update some documents successfully then not find
-   * them.
-   * <p>
-   * At least one I've seen (Erick Erickson) cannot be cured by waiting on the client side.
-   * <p>
-   * This is a horrible hack, but until we fix the underlying cause using it will reduce the noise from tests. Once the
-   * root cause of SOLR-11035 is found, this should be removed.
-   * <p>
-   * I don't mind the extra commits (why do two?) as this should be pretty rare.
-   *
-   * This test fails 10% - 15% of the time without using this method, especially if you @Ignore all the other
-   * tests in that suite.
-   *
-   * ant test  -Dtestcase=DocValuesNotIndexedTest -Dtests.method=testGroupingDVOnly
-   * -Dtests.seed=54688F608E614440 -Dtests.slow=true -Dtests.locale=nl-BE
-   * -Dtests.timezone=America/North_Dakota/Beulah -Dtests.asserts=true -Dtests.file.encoding=ISO-8859-1
-   *
-   * This only really works for adding documents. The test at the top of the method will succeed for an update of
-   * an existing doc and nothing will be done. If that becomes necessary we should probably create a new method
-   * that takes a docID, field and value.
-   *
-   * @param client - the client that we'll use to send the request
-   * @param collection - the target collection we'll add and remove the doc from
-   * @param idField - the uniqueKey for this collection. This MUST be a string
-   * @param expectedDocCount - numFound for the query
-   * @param query - The Solr query to check for expectedDocCount.
-   * @param tag - additional information to display on a failure. Often class.method is useful.
-   */
-
-  public static void Solr11035BandAid(SolrClient client, String collection, String idField,
-                                      long expectedDocCount, String query,
-                                      String tag) throws IOException, SolrServerException {
-
-    Solr11035BandAid(client, collection, idField, expectedDocCount, query, tag, false);
-  }
-
-  // Pass true for failAnyway to have this bandaid fail if
-  // 1> it had to attempt the repair
-  // 2> it would have successfully repaired it
-  //
-  // This is useful for verifying that SOLR-11035.
-  //
-  public static void Solr11035BandAid(SolrClient client, String collection, String idField,
-                                      long expectedDocCount, String query, String tag,
-                                      boolean failAnyway) throws IOException, SolrServerException {
-
-    final SolrQuery solrQuery = new SolrQuery(query);
-    QueryResponse rsp = client.query(collection, solrQuery);
-    long found = rsp.getResults().getNumFound();
-
-    if (found == expectedDocCount) {
-      return;
-    }
-
-    // OK, our counts differ. Insert a document _guaranteed_ to be unique, then delete it so whatever is counting
-    // anything has the correct counts.
-    log.warn("Solr11035BandAid attempting repair, found is {}, expected is {}", found, expectedDocCount);
-
-    String bogusID = java.util.UUID.randomUUID().toString();
-    SolrInputDocument bogus = new SolrInputDocument();
-    bogus.addField(idField, bogusID);
-
-    // Add the bogus doc
-    new UpdateRequest().add(bogus).commit(client, collection);
-
-    // Let's spin until we find the doc.
-    checkUniqueDoc(client, collection, idField, bogusID, true);
-
-    // Then remove it, we should be OK now since new searchers have been opened.
-    new UpdateRequest().deleteById(bogusID).commit(client, collection);
-
-    // Now spin until the doc is gone.
-    checkUniqueDoc(client, collection, idField, bogusID, false);
-
-    // At this point we're absolutely, totally, positive that a new searcher has been opened, so go ahead and check
-    // the actual condition.
-    rsp = client.query(collection, solrQuery);
-    found = rsp.getResults().getNumFound();
-
-    if (found != expectedDocCount) {
-      // It's good to see the total response. NOTE: some larger responses are over 10240,
-      // so change the pattern in log4j2.xml if you need to see the whole thing.
-      log.error("Dumping response" + rsp.toString());
-      assertEquals("Solr11035BandAid failed, counts differ after updates:", found, expectedDocCount);
-    } else if (failAnyway) {
-      fail("Solr11035BandAid failAnyway == true, would have successfully repaired the collection: '" + collection
-          + "' extra info: '" + tag + "'");
-    } else {
-      log.warn("Solr11035BandAid, repair successful");
-    }
-  }
-  // Helper for bandaid
-  private static void checkUniqueDoc(SolrClient client, String collection, String idField, String id, boolean shouldBeThere) throws IOException, SolrServerException {
-    TimeOut timeOut = new TimeOut(100, TimeUnit.SECONDS, TimeSource.NANO_TIME);
-    final SolrQuery solrQuery = new SolrQuery(idField + ":" + id);
-
-    while (!timeOut.hasTimedOut()) {
-      QueryResponse rsp = client.query(collection, solrQuery);
-      long found = rsp.getResults().getNumFound();
-      if (shouldBeThere && found == 1) {
-        return;
-      }
-      if (shouldBeThere == false && found == 0) {
-        return;
-      }
-      log.warn("Solr11035BandAid should have succeeded in checkUniqueDoc, shouldBeThere == {}, numFound = {}. Will try again after 250 ms sleep", shouldBeThere, found);
-      try {
-        Thread.sleep(250);
-      } catch (InterruptedException e) {
-        return; // just bail
-      }
-    }
-
-  }
 }