SOLR-11035: (at least) 2 distinct failures possible when clients attempt searches during SolrCore reload

This commit is contained in:
Erick Erickson 2020-02-21 10:57:16 -05:00
parent 88dd1c3f3d
commit 292bed0c42
4 changed files with 3 additions and 142 deletions

View File

@ -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.

View File

@ -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) {

View File

@ -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, "*:*"));

View File

@ -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
}
}
}
}