SOLR-14184: Internal 'test' variable DirectUpdateHandler2.commitOnClose has been removed and replaced with TestInjection.skipIndexWriterCommitOnClose

This commit is contained in:
Chris Hostetter 2020-01-16 11:47:06 -07:00
parent 94ee211366
commit 5f2d7c4855
14 changed files with 102 additions and 92 deletions

View File

@ -239,6 +239,10 @@ Other Changes
* SOLR-14169: Fix 20 Resource Leak warnings in SolrJ's apache/solr/common (Andras Salamon via Tomás Fernández Löbbe)
* SOLR-14184: Internal 'test' variable DirectUpdateHandler2.commitOnClose has been removed and replaced with
TestInjection.skipIndexWriterCommitOnClose. Users that modified DUH2.commitOnClose in test cases for custom
plugins/modicitations should now use TestInjection instead. (hossman)
================== 8.4.1 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -814,13 +814,11 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
}
}
public static volatile boolean commitOnClose = true; // TODO: make this a real config option or move it to TestInjection
// IndexWriterCloser interface method - called from solrCoreState.decref(this)
@Override
public void closeWriter(IndexWriter writer) throws IOException {
log.trace("closeWriter({}): ulog={}", writer, ulog);
assert TestInjection.injectNonGracefullClose(core.getCoreContainer());
boolean clearRequestInfo = false;
@ -832,18 +830,21 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp)); // important for debugging
}
try {
if (!commitOnClose) {
if (TestInjection.injectSkipIndexWriterCommitOnClose(writer)) {
// if this TestInjection triggers, we do some simple rollback()
// (which closes the underlying IndexWriter) and then return immediately
log.warn("Skipping commit for IndexWriter.close() due to TestInjection");
if (writer != null) {
writer.rollback();
}
// we shouldn't close the transaction logs either, but leaving them open
// means we can't delete them on windows (needed for tests)
if (ulog != null) ulog.close(false);
return;
}
// do a commit before we quit?
boolean tryToCommit = writer != null && ulog != null && ulog.hasUncommittedChanges()
&& ulog.getState() == UpdateLog.State.ACTIVE;
@ -852,8 +853,9 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
solrCoreState.getCommitLock().lock();
try {
try {
log.info("Committing on IndexWriter.close() {}.",
(tryToCommit ? "" : " ... SKIPPED (unnecessary)"));
if (tryToCommit) {
log.info("Committing on IndexWriter close.");
CommitUpdateCommand cmd = new CommitUpdateCommand(req, false);
cmd.openSearcher = false;
cmd.waitSearcher = false;

View File

@ -103,7 +103,7 @@ public class TestInjection {
}
public volatile static String nonGracefullClose = null;
public volatile static String failReplicaRequests = null;
public volatile static String failUpdateRequests = null;
@ -146,6 +146,15 @@ public class TestInjection {
public volatile static boolean failInExecutePlanAction = false;
/**
* Defaults to <code>false</code>, If set to <code>true</code>,
* then {@link #injectSkipIndexWriterCommitOnClose} will return <code>true</code>
*
* @see #injectSkipIndexWriterCommitOnClose
* @see org.apache.solr.update.DirectUpdateHandler2#closeWriter
*/
public volatile static boolean skipIndexWriterCommitOnClose = false;
public volatile static boolean uifOutOfMemoryError = false;
private volatile static CountDownLatch notifyPauseForeverDone = new CountDownLatch(1);
@ -176,6 +185,7 @@ public class TestInjection {
delayBeforeSlaveCommitRefresh = null;
delayInExecutePlanAction = null;
failInExecutePlanAction = false;
skipIndexWriterCommitOnClose = false;
uifOutOfMemoryError = false;
notifyPauseForeverDone();
newSearcherHooks.clear();
@ -278,6 +288,21 @@ public class TestInjection {
return true;
}
/**
* Returns the value of {@link #skipIndexWriterCommitOnClose}.
*
* @param indexWriter used only for logging
* @see #skipIndexWriterCommitOnClose
* @see org.apache.solr.update.DirectUpdateHandler2#closeWriter
*/
public static boolean injectSkipIndexWriterCommitOnClose(Object indexWriter) {
if (skipIndexWriterCommitOnClose) {
log.info("Inject failure: skipIndexWriterCommitOnClose={}: {}",
skipIndexWriterCommitOnClose, indexWriter);
}
return skipIndexWriterCommitOnClose;
}
public static boolean injectFailReplicaRequests() {
if (failReplicaRequests != null) {
Random rand = random();

View File

@ -57,10 +57,10 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.BadHdfsThreadsFilter;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.junit.AfterClass;
import org.junit.Before;
@ -146,10 +146,10 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
public void test() throws Exception {
try {
// to keep uncommitted docs during failover
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
testBasics();
} finally {
DirectUpdateHandler2.commitOnClose = true;
TestInjection.reset();
if (DEBUG) {
super.printLayout();
}

View File

@ -36,9 +36,9 @@ import org.apache.solr.common.cloud.ClusterStateUtil;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.core.SolrCore;
import org.apache.solr.metrics.SolrMetricManager;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.update.UpdateShardHandler;
import org.apache.solr.util.TestInjection;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
@ -90,6 +90,7 @@ public class TestCloudRecovery extends SolrCloudTestCase {
@After
public void afterTest() throws Exception {
TestInjection.reset(); // do after every test, don't wait for AfterClass
shutdownCluster();
}
@ -97,7 +98,7 @@ public class TestCloudRecovery extends SolrCloudTestCase {
// commented 4-Sep-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 20-Jul-2018
public void leaderRecoverFromLogOnStartupTest() throws Exception {
AtomicInteger countReplayLog = new AtomicInteger(0);
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
UpdateLog.testing_logReplayFinishHook = countReplayLog::incrementAndGet;
CloudSolrClient cloudClient = cluster.getSolrClient();
@ -163,7 +164,7 @@ public class TestCloudRecovery extends SolrCloudTestCase {
// commented out on: 17-Feb-2019 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 14-Oct-2018
public void corruptedLogTest() throws Exception {
AtomicInteger countReplayLog = new AtomicInteger(0);
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
UpdateLog.testing_logReplayFinishHook = countReplayLog::incrementAndGet;
CloudSolrClient cloudClient = cluster.getSolrClient();

View File

@ -40,7 +40,6 @@ import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.junit.After;
@ -64,11 +63,11 @@ public class TestTlogReplayVsRecovery extends SolrCloudTestCase {
// we also want to ensure that our leader doesn't do a "Commit on close"
//
// TODO: once SOLR-13486 is fixed, we should randomize this...
private static final boolean TEST_VALUE_FOR_COMMIT_ON_CLOSE = false;
private static final boolean TEST_VALUE_FOR_SKIP_COMMIT_ON_CLOSE = true;
@Before
public void setupCluster() throws Exception {
DirectUpdateHandler2.commitOnClose = TEST_VALUE_FOR_COMMIT_ON_CLOSE;
TestInjection.skipIndexWriterCommitOnClose = TEST_VALUE_FOR_SKIP_COMMIT_ON_CLOSE;
System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
System.setProperty("solr.ulog.numRecordsToKeep", "1000");
@ -99,7 +98,6 @@ public class TestTlogReplayVsRecovery extends SolrCloudTestCase {
@After
public void tearDownCluster() throws Exception {
TestInjection.reset();
DirectUpdateHandler2.commitOnClose = true;
if (null != proxies) {
for (SocketProxy proxy : proxies.values()) {
@ -163,8 +161,8 @@ public class TestTlogReplayVsRecovery extends SolrCloudTestCase {
addDocs(false, uncommittedDocs, committedDocs + 1);
log.info("Stopping leader node...");
assertEquals("Something broke our expected commitOnClose",
TEST_VALUE_FOR_COMMIT_ON_CLOSE, DirectUpdateHandler2.commitOnClose);
assertEquals("Something broke our expected skipIndexWriterCommitOnClose",
TEST_VALUE_FOR_SKIP_COMMIT_ON_CLOSE, TestInjection.skipIndexWriterCommitOnClose);
NODE0.stop();
cluster.waitForJettyToStop(NODE0);

View File

@ -62,7 +62,6 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.SolrCore;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.update.SolrIndexWriter;
import org.apache.solr.util.RefCounted;
import org.apache.solr.util.TestInjection;
@ -568,10 +567,13 @@ public class TestTlogReplica extends SolrCloudTestCase {
log.info("Min RF not achieved yet. retrying");
}
checkRTG(3,7, cluster.getJettySolrRunners());
DirectUpdateHandler2.commitOnClose = false;
solrRunner.stop();
waitForState("Replica still up", collectionName, activeReplicaCount(0,1,0));
DirectUpdateHandler2.commitOnClose = true;
try {
TestInjection.skipIndexWriterCommitOnClose = true;
solrRunner.stop();
waitForState("Replica still up", collectionName, activeReplicaCount(0,1,0));
} finally {
TestInjection.skipIndexWriterCommitOnClose = false;
}
solrRunner.start();
waitForState("Replica didn't recover", collectionName, activeReplicaCount(0,2,0));
waitForNumDocsInAllReplicas(5, getNonLeaderReplias(collectionName), 10); //timeout for stale collection state

View File

@ -34,8 +34,8 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.SolrCore;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.junit.Test;
@ -217,7 +217,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
waitForRecoveriesToFinish("unloadcollection", zkStateReader, false);
// so that we start with some versions when we reload...
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
try (HttpSolrClient addClient = getHttpSolrClient(jettys.get(2).getBaseUrl() + "/unloadcollection_shard1_replica3", 30000)) {
@ -290,8 +290,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
zkStateReader.getLeaderRetry("unloadcollection", "shard1", 15000);
// set this back
DirectUpdateHandler2.commitOnClose = true;
TestInjection.skipIndexWriterCommitOnClose = false; // set this back
assertTrue(CollectionAdminRequest
.addReplicaToShard("unloadcollection", "shard1")
.setCoreName(leaderProps.getCoreName())

View File

@ -43,7 +43,6 @@ import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.servlet.SolrDispatchFilter;
import org.apache.solr.update.AddUpdateCommand;
import org.apache.solr.update.CommitUpdateCommand;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.update.UpdateHandler;
import org.apache.solr.util.ReadOnlyCoresLocator;
import org.junit.BeforeClass;
@ -798,8 +797,6 @@ public class TestLazyCores extends SolrTestCaseJ4 {
// Cores 2, 3, 6, 7, 8, 9 are transient
@Test
public void testNoCommit() throws Exception {
DirectUpdateHandler2.commitOnClose = true;
CoreContainer cc = init();
String[] coreList = new String[]{
"collection2",

View File

@ -26,13 +26,13 @@ import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.update.AddUpdateCommand;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.update.UpdateHandler;
import org.apache.solr.update.processor.DistributedUpdateProcessorFactory;
import org.apache.solr.update.processor.UpdateRequestProcessor;
import org.apache.solr.update.processor.UpdateRequestProcessorChain;
import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
import org.apache.solr.util.TestInjection;
import org.junit.BeforeClass;
import org.junit.Test;
@ -70,7 +70,7 @@ public class TestSchemalessBufferedUpdates extends SolrTestCaseJ4 {
@Test
public void test() throws Exception {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
UpdateLog.testing_logReplayHook = () -> {
@ -118,7 +118,7 @@ public class TestSchemalessBufferedUpdates extends SolrTestCaseJ4 {
assertU(commit());
assertJQ(req("q", "*:*"), "/response/numFound==2");
} finally {
DirectUpdateHandler2.commitOnClose = true;
TestInjection.reset();
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
req().close();

View File

@ -46,10 +46,10 @@ import org.apache.solr.common.util.Utils;
import org.apache.solr.metrics.SolrMetricManager;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.schema.IndexSchema;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.update.UpdateHandler;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.junit.After;
import org.junit.Before;
@ -88,6 +88,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
@After
public void afterTest() {
TestInjection.reset(); // do after every test, don't wait for AfterClass
if (savedFactory == null) {
System.clearProperty("solr.directoryFactory");
} else {
@ -107,7 +108,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
public void stressLogReplay() throws Exception {
final int NUM_UPDATES = 150;
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -176,7 +177,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
"/response/docs==[{'val_i_dvo':"+entry.getValue()+"}]");
}
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -187,7 +187,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -294,7 +294,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
assertEquals(UpdateLog.State.ACTIVE, h.getCore().getUpdateHandler().getUpdateLog().getState());
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -305,7 +304,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
public void testNewDBQAndDocMatchingOldDBQDuringLogReplay() throws Exception {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -366,7 +365,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
, "/response/docs==[{'id':'B0'}, {'id':'B3'}, {'id':'B5'}, {'id':'B6'}]");
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -483,7 +481,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -528,7 +526,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
} catch (Throwable thr) {
throw new Exception(thr);
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -538,7 +535,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
@Test
public void testBuffering() throws Exception {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -712,7 +709,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
assertEquals(0, bufferedOps.getValue().intValue());
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
@ -725,7 +721,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
@Test
public void testDropBuffered() throws Exception {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -858,7 +854,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
assertEquals(UpdateLog.State.ACTIVE, ulog.getState()); // leave each test method in a good state
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
@ -870,7 +865,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
@Test
public void testBufferedMultipleCalls() throws Exception {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -964,7 +959,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
assertEquals(UpdateLog.State.ACTIVE, ulog.getState()); // leave each test method in a good state
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
@ -1009,7 +1003,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
@Test
public void testExistOldBufferLog() throws Exception {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
SolrQueryRequest req = req();
UpdateHandler uhandler = req.getCore().getUpdateHandler();
@ -1083,7 +1077,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
assertJQ(req("qt","/get", "id", "Q7") ,"/doc/id==Q7");
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
@ -1127,7 +1120,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
// make sure that log isn't needlessly replayed after a clean close
@Test
public void testCleanShutdown() throws Exception {
DirectUpdateHandler2.commitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -1168,7 +1160,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
assertEquals(10, logReplay.availablePermits());
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
@ -1186,7 +1177,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
@Test
public void testRemoveOldLogs() throws Exception {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -1297,7 +1288,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
resetExceptionIgnores();
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -1310,7 +1300,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
@Test
public void testTruncatedLog() throws Exception {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -1368,7 +1358,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
assertJQ(req("qt","/get", "getVersions","3"), "/versions==["+v106+","+v105+","+v104+"]");
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -1381,7 +1370,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
@Test
public void testCorruptLog() throws Exception {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
File logDir = new File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
@ -1435,7 +1424,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
deleteLogs();
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -1447,7 +1435,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
@Test
public void testRecoveryMultipleLogs() throws Exception {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -1510,7 +1498,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
assertJQ(req("q","*:*") ,"/response/numFound==6");
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -1521,7 +1508,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -1637,7 +1624,6 @@ public class TestRecovery extends SolrTestCaseJ4 {
assertEquals(UpdateLog.State.ACTIVE, h.getCore().getUpdateHandler().getUpdateLog().getState());
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}

View File

@ -45,13 +45,14 @@ import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.update.HdfsUpdateLog;
import org.apache.solr.update.UpdateHandler;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
import org.apache.solr.util.BadHdfsThreadsFilter;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore;
@ -74,6 +75,11 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
private static String hdfsUri;
private static FileSystem fs;
@After
public void afterTest() {
TestInjection.reset(); // do after every test, don't wait for AfterClass
}
@BeforeClass
public static void beforeClass() throws Exception {
dfsCluster = HdfsTestUtil.setupClass(createTempDir().toFile().getAbsolutePath());
@ -137,7 +143,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
@Test
public void testLogReplay() throws Exception {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -222,7 +228,6 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
assertEquals(UpdateLog.State.ACTIVE, h.getCore().getUpdateHandler().getUpdateLog().getState());
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -230,7 +235,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
@Test
public void testBuffering() throws Exception {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -370,7 +375,6 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
assertEquals(UpdateLog.State.ACTIVE, ulog.getState()); // leave each test method in a good state
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
@ -381,7 +385,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
@Test
@Ignore("HDFS-3107: no truncate support yet")
public void testDropBuffered() throws Exception {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -493,7 +497,6 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
assertEquals(UpdateLog.State.ACTIVE, ulog.getState()); // leave each test method in a good state
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
@ -503,7 +506,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
@Test
public void testExistOldBufferLog() throws Exception {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
SolrQueryRequest req = req();
UpdateHandler uhandler = req.getCore().getUpdateHandler();
@ -564,7 +567,6 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
() -> h.getCore().getUpdateHandler().getUpdateLog().getState() == UpdateLog.State.ACTIVE);
assertJQ(req("qt","/get", "id", "Q7") ,"/doc/id==Q7");
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
@ -601,7 +603,6 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
// make sure that log isn't needlessly replayed after a clean close
@Test
public void testCleanShutdown() throws Exception {
DirectUpdateHandler2.commitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -639,7 +640,6 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
assertEquals(10, logReplay.availablePermits());
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
@ -656,7 +656,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
@Test
public void testRemoveOldLogs() throws Exception {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -750,7 +750,6 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
resetExceptionIgnores();
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -763,7 +762,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
@Test
public void testTruncatedLog() throws Exception {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -820,7 +819,6 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
assertJQ(req("qt","/get", "getVersions","3"), "/versions==[106,105,104]");
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -832,7 +830,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
@Test
public void testCorruptLog() throws Exception {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
String logDir = h.getCore().getUpdateHandler().getUpdateLog().getLogDir();
@ -881,7 +879,6 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
deleteLogs();
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -891,7 +888,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
@Test
public void testRecoveryMultipleLogs() throws Exception {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -954,7 +951,6 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
assertJQ(req("q","*:*") ,"/response/numFound==6");
} finally {
DirectUpdateHandler2.commitOnClose = true;
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}

View File

@ -31,6 +31,7 @@ import org.apache.lucene.util.LuceneTestCase.Nightly;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.util.TestInjection;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -415,7 +416,7 @@ public class CdcrUpdateLogTest extends SolrTestCaseJ4 {
public void testClosingOutputStreamAfterLogReplay() throws Exception {
this.clearCore();
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplay = new Semaphore(0);
final Semaphore logReplayFinish = new Semaphore(0);
@ -464,7 +465,7 @@ public class CdcrUpdateLogTest extends SolrTestCaseJ4 {
assertTrue(ulog.logs.peekLast().endsWithCommit());
ulog.logs.peekLast().decref();
} finally {
DirectUpdateHandler2.commitOnClose = true; // reset
TestInjection.skipIndexWriterCommitOnClose = false; // reset
UpdateLog.testing_logReplayHook = null;
UpdateLog.testing_logReplayFinishHook = null;
}
@ -641,7 +642,7 @@ public class CdcrUpdateLogTest extends SolrTestCaseJ4 {
@Test
public void testGetNumberOfRemainingRecords() throws Exception {
try {
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
final Semaphore logReplayFinish = new Semaphore(0);
UpdateLog.testing_logReplayFinishHook = () -> logReplayFinish.release();
@ -685,7 +686,7 @@ public class CdcrUpdateLogTest extends SolrTestCaseJ4 {
addDocs(10, start, versions);
assertEquals(10, reader.getNumberOfRemainingRecords());
} finally {
DirectUpdateHandler2.commitOnClose = true;
TestInjection.skipIndexWriterCommitOnClose = false; // reset
UpdateLog.testing_logReplayFinishHook = null;
}
}

View File

@ -43,9 +43,9 @@ import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrCore;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.RTimer;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
@ -482,7 +482,7 @@ public class ChaosMonkey {
if (chaosRandom.nextBoolean()) {
monkeyLog("Jetty will not commit on close");
DirectUpdateHandler2.commitOnClose = false;
TestInjection.skipIndexWriterCommitOnClose = true;
}
this.aggressivelyKillLeaders = killLeaders;
@ -533,8 +533,7 @@ public class ChaosMonkey {
Thread.currentThread().interrupt();
}
runTimer.stop();
DirectUpdateHandler2.commitOnClose = true;
TestInjection.skipIndexWriterCommitOnClose = false;
double runtime = runTimer.getTime()/1000.0f;
if (runtime > NO_STOP_WARN_TIME && stops.get() == 0) {