mirror of https://github.com/apache/lucene.git
SOLR-6231: Harden the RollingRestartTest
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1612499 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a265a8ab9c
commit
1c6508b284
|
@ -32,6 +32,8 @@ import java.util.concurrent.TimeUnit;
|
|||
public class RollingRestartTest extends AbstractFullDistribZkTestBase {
|
||||
public static Logger log = LoggerFactory.getLogger(ChaosMonkeyNothingIsSafeTest.class);
|
||||
|
||||
private static final long MAX_WAIT_TIME = TimeUnit.NANOSECONDS.convert(300, TimeUnit.SECONDS);
|
||||
|
||||
public RollingRestartTest() {
|
||||
fixShardCount = true;
|
||||
sliceCount = 2;
|
||||
|
@ -83,7 +85,7 @@ public class RollingRestartTest extends AbstractFullDistribZkTestBase {
|
|||
overseerDesignates.add(cloudJettys.get(n));
|
||||
}
|
||||
|
||||
waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, 60);
|
||||
waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, MAX_WAIT_TIME);
|
||||
|
||||
cloudClient.getZkStateReader().getZkClient().printLayoutToStdOut();
|
||||
|
||||
|
@ -93,18 +95,20 @@ public class RollingRestartTest extends AbstractFullDistribZkTestBase {
|
|||
for (CloudJettyRunner cloudJetty : overseerDesignates) {
|
||||
log.info("Restarting {}", cloudJetty);
|
||||
chaosMonkey.stopJetty(cloudJetty);
|
||||
boolean success = waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, 60);
|
||||
boolean success = waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, MAX_WAIT_TIME);
|
||||
if (!success) {
|
||||
leader = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
|
||||
if(leader == null) log.error("NOOVERSEER election queue is :"+ OverseerCollectionProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient()));
|
||||
if (leader == null)
|
||||
log.error("NOOVERSEER election queue is :" + OverseerCollectionProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient()));
|
||||
fail("No overseer designate as leader found after restart #" + (i + 1) + ": " + leader);
|
||||
}
|
||||
assertTrue("Unable to restart (#"+i+"): " + cloudJetty,
|
||||
chaosMonkey.start(cloudJetty.jetty));
|
||||
success = waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, 60);
|
||||
assertTrue("Unable to restart (#" + i + "): " + cloudJetty,
|
||||
ChaosMonkey.start(cloudJetty.jetty));
|
||||
success = waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, MAX_WAIT_TIME);
|
||||
if (!success) {
|
||||
leader = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
|
||||
if(leader == null) log.error("NOOVERSEER election queue is :"+ OverseerCollectionProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient()));
|
||||
if (leader == null)
|
||||
log.error("NOOVERSEER election queue is :" + OverseerCollectionProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient()));
|
||||
fail("No overseer leader found after restart #" + (i + 1) + ": " + leader);
|
||||
}
|
||||
}
|
||||
|
@ -117,24 +121,35 @@ public class RollingRestartTest extends AbstractFullDistribZkTestBase {
|
|||
cloudClient.getZkStateReader().getZkClient().printLayoutToStdOut();
|
||||
}
|
||||
|
||||
static boolean waitUntilOverseerDesignateIsLeader(SolrZkClient testZkClient, List<String> overseerDesignates, int timeoutInSeconds) throws KeeperException, InterruptedException {
|
||||
static boolean waitUntilOverseerDesignateIsLeader(SolrZkClient testZkClient, List<String> overseerDesignates, long timeoutInNanos) throws KeeperException, InterruptedException {
|
||||
long now = System.nanoTime();
|
||||
long timeout = now + TimeUnit.NANOSECONDS.convert(timeoutInSeconds, TimeUnit.SECONDS);
|
||||
long maxTimeout = now + timeoutInNanos; // the maximum amount of time we're willing to wait to see the designate as leader
|
||||
long timeout = now + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);
|
||||
boolean firstTime = true;
|
||||
int stableCheckTimeout = 2000;
|
||||
while (System.nanoTime() < timeout) {
|
||||
String oldleader = null;
|
||||
while (System.nanoTime() < timeout && System.nanoTime() < maxTimeout) {
|
||||
String newLeader = OverseerCollectionProcessor.getLeaderNode(testZkClient);
|
||||
if (newLeader != null && !newLeader.equals(oldleader)) {
|
||||
// the leaders have changed, let's move the timeout further
|
||||
timeout = System.nanoTime() + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);
|
||||
log.info("oldLeader={} newLeader={} - Advancing timeout to: {}", oldleader, newLeader, timeout);
|
||||
oldleader = newLeader;
|
||||
}
|
||||
if (!overseerDesignates.contains(newLeader)) {
|
||||
Thread.sleep(500);
|
||||
} else {
|
||||
if (firstTime) {
|
||||
if (firstTime) {
|
||||
firstTime = false;
|
||||
Thread.sleep(stableCheckTimeout);
|
||||
} else {
|
||||
} else {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (System.nanoTime() < maxTimeout) {
|
||||
log.error("Max wait time exceeded");
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue