mirror of https://github.com/apache/lucene.git
SOLR-5417: fail if the chaosmonkey runs for over 20 seconds and doesn't kill a jetty
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1545125 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
262e0c4190
commit
56a116aae8
|
@ -144,7 +144,7 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
|
|||
if (RUN_LENGTH != -1) {
|
||||
runLength = RUN_LENGTH;
|
||||
} else {
|
||||
int[] runTimes = new int[] {5000,6000,10000,15000,15000,30000,30000,45000,90000,120000};
|
||||
int[] runTimes = new int[] {5000,6000,10000,15000,25000,30000,30000,45000,90000,120000};
|
||||
runLength = runTimes[random().nextInt(runTimes.length - 1)];
|
||||
}
|
||||
|
||||
|
|
|
@ -104,7 +104,7 @@ public class ChaosMonkeySafeLeaderTest extends AbstractFullDistribZkTestBase {
|
|||
if (RUN_LENGTH != -1) {
|
||||
runLength = RUN_LENGTH;
|
||||
} else {
|
||||
int[] runTimes = new int[] {5000,6000,10000,15000,15000,30000,30000,45000,90000,120000};
|
||||
int[] runTimes = new int[] {5000,6000,10000,25000,27000,30000,30000,45000,90000,120000};
|
||||
runLength = runTimes[random().nextInt(runTimes.length - 1)];
|
||||
}
|
||||
try {
|
||||
|
|
|
@ -23,6 +23,8 @@ import java.util.Map;
|
|||
import java.util.Random;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.solr.client.solrj.SolrServer;
|
||||
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
|
||||
|
@ -74,7 +76,7 @@ public class ChaosMonkey {
|
|||
private boolean causeConnectionLoss;
|
||||
private boolean aggressivelyKillLeaders;
|
||||
private Map<String,CloudJettyRunner> shardToLeaderJetty;
|
||||
private long startTime;
|
||||
private volatile long startTime;
|
||||
|
||||
private Thread monkeyThread;
|
||||
|
||||
|
@ -519,6 +521,10 @@ public class ChaosMonkey {
|
|||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
float runtime = (System.currentTimeMillis() - startTime)/1000.0f;
|
||||
if (runtime > 20 && stops.get() == 0) {
|
||||
TestCase.fail("The Monkey ran for over 20 seconds and no jetties were stopped - this is worth investigating!");
|
||||
}
|
||||
}
|
||||
|
||||
public int getStarts() {
|
||||
|
|
Loading…
Reference in New Issue