mirror of https://github.com/apache/lucene.git
SOLR-11035: (at least) 2 distinct failures possible when clients attempt searches during SolrCore reload bandaid for DocValuesNotIndexedTest
This commit is contained in:
parent
7ba8bdca8f
commit
2a3bd8e864
|
@ -235,6 +235,9 @@ Improvements
|
|||
|
||||
* SOLR-13407: Reject update requests sent to non-routed multi collection aliases. (ab)
|
||||
|
||||
* SOLR-11035: (at least) 2 distinct failures possible when clients attempt searches during SolrCore reload,
|
||||
added test band-aid for DocValuesNotIndexedTest.
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -243,6 +243,7 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
|
|||
new UpdateRequest()
|
||||
.add(docs)
|
||||
.commit(client, COLLECTION);
|
||||
Solr11035BandAid(client, COLLECTION, "id", 4, "*:*");
|
||||
|
||||
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"});
|
||||
|
@ -286,6 +287,7 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
|
|||
.add(docs)
|
||||
.commit(client, COLLECTION);
|
||||
|
||||
Solr11035BandAid(client, COLLECTION, "id", 4, "*:*");
|
||||
// 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
|
||||
|
@ -321,17 +323,21 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
|
|||
doGroupingDvOnly(fieldsToTestGroupSortFirst, "boolGSF");
|
||||
doGroupingDvOnly(fieldsToTestGroupSortLast, "boolGSL");
|
||||
}
|
||||
private void doGroupingDvOnly(List<FieldProps> fieldProps, String boolName) throws IOException, SolrServerException {
|
||||
|
||||
private void doGroupingDvOnly(List<FieldProps> fieldProps, String boolName) throws IOException, SolrServerException {
|
||||
List<SolrInputDocument> docs = new ArrayList<>(50);
|
||||
for (int idx = 0; idx < 49; ++idx) {
|
||||
SolrInputDocument doc = new SolrInputDocument();
|
||||
doc.addField("id", idx);
|
||||
boolean doInc = ((idx % 7) == 0);
|
||||
|
||||
// Every 7th doc we bump a counter by some random amount
|
||||
for (FieldProps prop : fieldProps) {
|
||||
doc.addField(prop.getName(), prop.getValue(doInc));
|
||||
doc.addField(prop.getName(), prop.getValue((idx % 7) == 0));
|
||||
}
|
||||
docs.add(doc);
|
||||
|
||||
// Every fifth time through we add a doc with no values in any of the "fields of interest", so there shoule be
|
||||
// 10 docs with nulls
|
||||
if ((idx % 5) == 0) {
|
||||
doc = new SolrInputDocument();
|
||||
doc.addField("id", idx + 10_000);
|
||||
|
@ -345,6 +351,8 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
|
|||
.add(docs)
|
||||
.commit(client, COLLECTION);
|
||||
|
||||
Solr11035BandAid(client, COLLECTION,"id", 59, "*:*");
|
||||
|
||||
// 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) {
|
||||
|
||||
|
@ -356,7 +364,8 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
|
|||
"rows", "100",
|
||||
"group", "true",
|
||||
"group.field", prop.getName(),
|
||||
"group.limit", "100");
|
||||
"group.limit", "100",
|
||||
"group.sort", "id asc");
|
||||
|
||||
final QueryResponse rsp = client.query(COLLECTION, solrQuery);
|
||||
|
||||
|
|
|
@ -84,6 +84,9 @@ 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;
|
||||
|
@ -95,6 +98,8 @@ 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.util.ClientUtils;
|
||||
import org.apache.solr.cloud.IpTables;
|
||||
import org.apache.solr.cloud.MiniSolrCloudCluster;
|
||||
|
@ -2981,4 +2986,68 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
|
|||
= 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.
|
||||
*/
|
||||
|
||||
public static void Solr11035BandAid(SolrClient client, String collection, String idField,
|
||||
long expectedDocCount, String query) throws IOException, SolrServerException {
|
||||
final SolrQuery solrQuery = new SolrQuery(query);
|
||||
QueryResponse rsp = client.query(collection, solrQuery);
|
||||
long found = rsp.getResults().getNumFound();
|
||||
|
||||
if (rsp.getResults().getNumFound() == 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);
|
||||
|
||||
// Then remove it, we should be OK now since new searchers have been opened.
|
||||
new UpdateRequest().deleteById(bogusID).commit(client, collection);
|
||||
// Let's check again to see if we succeeded
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue