mirror of https://github.com/apache/lucene.git
SOLR-13032: harden MaxSizeAutoCommitTest to eliminate race conditions and eliminate use of arbitrary sleep
This commit is contained in:
parent
ca4555ed93
commit
868e005760
|
@ -326,4 +326,9 @@ public final class CommitTracker implements Runnable {
|
|||
public void setOpenSearcher(boolean openSearcher) {
|
||||
this.openSearcher = openSearcher;
|
||||
}
|
||||
|
||||
// only for testing - not thread safe
|
||||
public boolean hasPending() {
|
||||
return (null != pending && !pending.isDone());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,9 +17,13 @@
|
|||
|
||||
package org.apache.solr.update;
|
||||
|
||||
import java.lang.invoke.MethodHandles;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
|
||||
|
@ -27,16 +31,24 @@ import org.apache.solr.SolrTestCaseJ4;
|
|||
import org.apache.solr.client.solrj.util.ClientUtils;
|
||||
import org.apache.solr.common.params.MapSolrParams;
|
||||
import org.apache.solr.common.util.ContentStream;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.core.SolrEventListener;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.handler.UpdateRequestHandler;
|
||||
import org.apache.solr.request.SolrQueryRequestBase;
|
||||
import org.apache.solr.response.SolrQueryResponse;
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 {
|
||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||
|
||||
// Given an ID, returns an XML string for an "add document" request
|
||||
private static final Function<Integer, String> ADD_DOC_FN = (id) -> adoc("id", Integer.toString(id));
|
||||
|
@ -44,11 +56,14 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 {
|
|||
private static final Function<Integer, String> DELETE_DOC_FN = (id) -> delI(Integer.toString(id));
|
||||
// How long to sleep while checking for commits
|
||||
private static final int COMMIT_CHECKING_SLEEP_TIME_MS = 50;
|
||||
|
||||
// max TLOG file size
|
||||
private static final int MAX_FILE_SIZE = 1000;
|
||||
|
||||
private SolrCore core;
|
||||
private DirectUpdateHandler2 updateHandler;
|
||||
private CommitTracker hardCommitTracker;
|
||||
private UpdateRequestHandler updateRequestHandler;
|
||||
private MockEventListener monitor;
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
|
@ -56,63 +71,73 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 {
|
|||
initCore("solrconfig-tlog.xml", "schema.xml");
|
||||
core = h.getCore();
|
||||
updateHandler = (DirectUpdateHandler2) core.getUpdateHandler();
|
||||
|
||||
// we don't care about auto-commit's opening a new Searcher in this test, just skip it.
|
||||
updateHandler.softCommitTracker.setOpenSearcher(false);
|
||||
updateHandler.commitTracker.setOpenSearcher(false);
|
||||
|
||||
// we don't care about soft commit's at all
|
||||
updateHandler.softCommitTracker.setTimeUpperBound(-1);
|
||||
updateHandler.softCommitTracker.setDocsUpperBound(-1);
|
||||
updateHandler.softCommitTracker.setTLogFileSizeUpperBound(-1);
|
||||
|
||||
hardCommitTracker = updateHandler.commitTracker;
|
||||
// Only testing file-size based auto hard commits - disable other checks
|
||||
hardCommitTracker.setTimeUpperBound(-1);
|
||||
hardCommitTracker.setDocsUpperBound(-1);
|
||||
hardCommitTracker.setTLogFileSizeUpperBound(MAX_FILE_SIZE);
|
||||
|
||||
monitor = new MockEventListener();
|
||||
updateHandler.registerCommitCallback(monitor);
|
||||
|
||||
updateRequestHandler = new UpdateRequestHandler();
|
||||
updateRequestHandler.init( null );
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
monitor.assertSaneOffers();
|
||||
monitor.clear();
|
||||
super.tearDown();
|
||||
System.clearProperty("solr.ulog");
|
||||
deleteCore();
|
||||
}
|
||||
|
||||
@Test
|
||||
@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-13032")
|
||||
public void testAdds() throws Exception {
|
||||
int maxFileSizeBound = 1000;
|
||||
// Set max size bound
|
||||
hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound);
|
||||
|
||||
// Add a large number of docs - should trigger a commit
|
||||
int numDocsToAdd = 500;
|
||||
SolrQueryResponse updateResp = new SolrQueryResponse();
|
||||
|
||||
Assert.assertEquals("There have been no updates yet, so there shouldn't have been any commits", 0,
|
||||
hardCommitTracker.getCommitCount());
|
||||
hardCommitTracker.getCommitCount());
|
||||
|
||||
long tlogSizePreUpdates = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
|
||||
Assert.assertEquals("There have been no updates yet, so tlog should be empty", 0, tlogSizePreUpdates);
|
||||
|
||||
updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
|
||||
|
||||
// The long sleep is to allow for the triggered commit to finish
|
||||
waitForCommit(1000);
|
||||
|
||||
// Verify commit information
|
||||
Assert.assertTrue("At least one commit should have occurred", hardCommitTracker.getCommitCount() > 0);
|
||||
long tlogSizePostUpdates = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
|
||||
Assert.assertTrue("Current tlog size is larger than the max bound", tlogSizePostUpdates < maxFileSizeBound);
|
||||
// Add a large number of docs - should trigger a commit
|
||||
int numDocsToAdd = 500;
|
||||
SolrQueryResponse updateResp = new SolrQueryResponse();
|
||||
|
||||
monitor.doStuffAndExpectAtLeastOneCommit(hardCommitTracker, updateHandler, () -> {
|
||||
updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRedundantDeletes() throws Exception {
|
||||
int maxFileSizeBound = 1000;
|
||||
// Set max size bound
|
||||
hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound);
|
||||
|
||||
Assert.assertEquals("There have been no updates yet, so there shouldn't have been any commits", 0,
|
||||
hardCommitTracker.getCommitCount());
|
||||
|
||||
long tlogSizePreUpdates = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
|
||||
Assert.assertEquals("There have been no updates yet, so tlog should be empty", 0, tlogSizePreUpdates);
|
||||
|
||||
// Add docs
|
||||
int numDocsToAdd = 150;
|
||||
SolrQueryResponse updateResp = new SolrQueryResponse();
|
||||
updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
|
||||
waitForCommit(1000);
|
||||
|
||||
// Get the current commit info
|
||||
int commitCountPreDeletes = hardCommitTracker.getCommitCount();
|
||||
monitor.doStuffAndExpectAtLeastOneCommit(hardCommitTracker, updateHandler, () -> {
|
||||
updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
|
||||
});
|
||||
|
||||
|
||||
// Send a bunch of redundant deletes
|
||||
int numDeletesToSend = 500;
|
||||
|
@ -124,63 +149,36 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 {
|
|||
docs.add(delI(Integer.toString(docIdToDelete)));
|
||||
}
|
||||
batchSingleDeleteRequest.setContentStreams(toContentStreams(docs));
|
||||
|
||||
updateRequestHandler.handleRequest(batchSingleDeleteRequest, updateResp);
|
||||
|
||||
// The long sleep is to allow for the expected triggered commit to finish
|
||||
waitForCommit(1000);
|
||||
|
||||
// Verify commit information
|
||||
Assert.assertTrue("At least one commit should have occurred",
|
||||
hardCommitTracker.getCommitCount() > commitCountPreDeletes);
|
||||
long tlogSizePostDeletes = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
|
||||
Assert.assertTrue("Current tlog size is larger than the max bound", tlogSizePostDeletes < maxFileSizeBound);
|
||||
|
||||
monitor.doStuffAndExpectAtLeastOneCommit(hardCommitTracker, updateHandler, () -> {
|
||||
updateRequestHandler.handleRequest(batchSingleDeleteRequest, updateResp);
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-13032")
|
||||
public void testDeletes() throws Exception {
|
||||
int maxFileSizeBound = 1000;
|
||||
|
||||
// Set max size bound
|
||||
hardCommitTracker.setTLogFileSizeUpperBound(maxFileSizeBound);
|
||||
Assert.assertEquals("There have been no updates yet, so there shouldn't have been any commits", 0,
|
||||
hardCommitTracker.getCommitCount());
|
||||
|
||||
long tlogSizePreUpdates = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
|
||||
Assert.assertEquals("There have been no updates yet, so tlog should be empty", 0, tlogSizePreUpdates);
|
||||
|
||||
// Add docs
|
||||
int numDocsToAdd = 500;
|
||||
SolrQueryResponse updateResp = new SolrQueryResponse();
|
||||
updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
|
||||
waitForCommit(1000);
|
||||
|
||||
// Get the current commit info
|
||||
int commitCountPreDeletes = hardCommitTracker.getCommitCount();
|
||||
|
||||
|
||||
monitor.doStuffAndExpectAtLeastOneCommit(hardCommitTracker, updateHandler, () -> {
|
||||
updateRequestHandler.handleRequest(constructBatchAddDocRequest(0, numDocsToAdd), updateResp);
|
||||
});
|
||||
|
||||
// Delete all documents - should trigger a commit
|
||||
updateRequestHandler.handleRequest(constructBatchDeleteDocRequest(0, numDocsToAdd), updateResp);
|
||||
|
||||
// The long sleep is to allow for the expected triggered commit to finish
|
||||
waitForCommit(1000);
|
||||
|
||||
// Verify commit information
|
||||
Assert.assertTrue("At least one commit should have occurred",
|
||||
hardCommitTracker.getCommitCount() > commitCountPreDeletes);
|
||||
long tlogSizePostDeletes = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
|
||||
Assert.assertTrue("Current tlog size is larger than the max bound", tlogSizePostDeletes < maxFileSizeBound);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sleeps in increments of COMMIT_CHECKING_SLEEP_TIME_MS while checking to see if a commit completed. If it did,
|
||||
* then return. If not, continue this cycle for at most the amount of time specified
|
||||
* @param maxTotalWaitTimeMillis the max amount of time (in ms) to wait/check for a commit
|
||||
*/
|
||||
private void waitForCommit(long maxTotalWaitTimeMillis) throws Exception {
|
||||
long startTimeNanos = System.nanoTime();
|
||||
long maxTotalWaitTimeNanos = TimeUnit.MILLISECONDS.toNanos(maxTotalWaitTimeMillis);
|
||||
while (System.nanoTime() - startTimeNanos < maxTotalWaitTimeNanos) {
|
||||
Thread.sleep(COMMIT_CHECKING_SLEEP_TIME_MS);
|
||||
if (!updateHandler.getUpdateLog().hasUncommittedChanges()) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
monitor.doStuffAndExpectAtLeastOneCommit(hardCommitTracker, updateHandler, () -> {
|
||||
updateRequestHandler.handleRequest(constructBatchDeleteDocRequest(0, numDocsToAdd), updateResp);
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -232,4 +230,93 @@ public class MaxSizeAutoCommitTest extends SolrTestCaseJ4 {
|
|||
}
|
||||
return streams;
|
||||
}
|
||||
|
||||
private static final class MockEventListener implements SolrEventListener {
|
||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||
|
||||
public MockEventListener() {
|
||||
/* No-Op */
|
||||
}
|
||||
|
||||
// use capacity bound Queue just so we're sure we don't OOM
|
||||
public final BlockingQueue<Long> hard = new LinkedBlockingQueue<>(1000);
|
||||
|
||||
// if non enpty, then at least one offer failed (queues full)
|
||||
private StringBuffer fail = new StringBuffer();
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {}
|
||||
|
||||
@Override
|
||||
public void newSearcher(SolrIndexSearcher newSearcher, SolrIndexSearcher currentSearcher) {
|
||||
// No-Op
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postCommit() {
|
||||
Long now = System.nanoTime();
|
||||
if (!hard.offer(now)) fail.append(", hardCommit @ " + now);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postSoftCommit() {
|
||||
// No-Op
|
||||
}
|
||||
|
||||
public void clear() {
|
||||
hard.clear();
|
||||
fail.setLength(0);
|
||||
}
|
||||
|
||||
public void doStuffAndExpectAtLeastOneCommit(final CommitTracker commitTracker,
|
||||
final DirectUpdateHandler2 updateHandler,
|
||||
final Runnable stuff) throws InterruptedException {
|
||||
assertSaneOffers();
|
||||
|
||||
final int POLL_TIME = 5;
|
||||
final TimeUnit POLL_UNIT = TimeUnit.SECONDS;
|
||||
|
||||
final int preAutoCommitCount = commitTracker.getCommitCount();
|
||||
log.info("Auto-Commit count prior to doing work: {}", preAutoCommitCount);
|
||||
stuff.run();
|
||||
log.info("Work Completed");
|
||||
|
||||
int numIters = 0;
|
||||
Long lastPostCommitTimeStampSeen = null;
|
||||
final long startTimeNanos = System.nanoTime();
|
||||
final long cutOffTime = startTimeNanos + TimeUnit.SECONDS.toNanos(300);
|
||||
while (System.nanoTime() < cutOffTime) {
|
||||
numIters++;
|
||||
log.info("Polling at most {} {} for expected (post-)commit#{}", POLL_TIME, POLL_UNIT, numIters);
|
||||
lastPostCommitTimeStampSeen = hard.poll(POLL_TIME, POLL_UNIT);
|
||||
assertNotNull("(post-)commit#" + numIters + " didn't occur in allowed time frame",
|
||||
lastPostCommitTimeStampSeen);
|
||||
|
||||
synchronized (commitTracker) {
|
||||
final int currentAutoCommitCount = commitTracker.getCommitCount() - preAutoCommitCount;
|
||||
final long currentFileSize = updateHandler.getUpdateLog().getCurrentLogSizeFromStream();
|
||||
if ((currentFileSize < MAX_FILE_SIZE) &&
|
||||
(currentAutoCommitCount == numIters) &&
|
||||
( ! commitTracker.hasPending() )) {
|
||||
// if all of these condiions are met, then we should be completely done
|
||||
assertSaneOffers(); // last minute sanity check
|
||||
return;
|
||||
}
|
||||
// else: log & loop...
|
||||
log.info("(Auto-)commits triggered: {}; (post-)commits seen: {}; current tlog file size: {}",
|
||||
currentAutoCommitCount, numIters, currentFileSize);
|
||||
}
|
||||
}
|
||||
|
||||
// if we didn't return already, then we ran out of time
|
||||
fail("Exhausted cut off time polling for post-commit events (got " + numIters + ")");
|
||||
}
|
||||
|
||||
public void assertSaneOffers() {
|
||||
assertEquals("Failure of MockEventListener" + fail.toString(),
|
||||
0, fail.length());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue