SOLR-6291: RollingRestartTest is too slow.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1617482 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2014-08-12 14:14:33 +00:00
parent f6d0fe58ed
commit 693a0c0291
2 changed files with 40 additions and 21 deletions

View File

@ -17,6 +17,7 @@ package org.apache.solr.cloud;
* limitations under the License.
*/
import org.apache.commons.collections.CollectionUtils;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.params.CollectionParams;
import org.apache.zookeeper.KeeperException;
@ -37,7 +38,7 @@ public class RollingRestartTest extends AbstractFullDistribZkTestBase {
public RollingRestartTest() {
fixShardCount = true;
sliceCount = 2;
shardCount = 16;
shardCount = TEST_NIGHTLY ? 16 : 2;
}
@Before
@ -73,46 +74,58 @@ public class RollingRestartTest extends AbstractFullDistribZkTestBase {
cloudClient.getZkStateReader().getZkClient().printLayoutToStdOut();
int numOverseers = 3;
int numDesignateOverseers = TEST_NIGHTLY ? 16 : 2;
numDesignateOverseers = Math.max(shardCount, numDesignateOverseers);
List<String> designates = new ArrayList<>();
List<CloudJettyRunner> overseerDesignates = new ArrayList<>();
for (int i = 0; i < numOverseers; i++) {
List<CloudJettyRunner> designateJettys = new ArrayList<>();
for (int i = 0; i < numDesignateOverseers; i++) {
int n = random().nextInt(shardCount);
String nodeName = cloudJettys.get(n).nodeName;
log.info("Chose {} as overseer designate", nodeName);
invokeCollectionApi(CollectionParams.ACTION, CollectionParams.CollectionAction.ADDROLE.toLower(), "role", "overseer", "node", nodeName);
designates.add(nodeName);
overseerDesignates.add(cloudJettys.get(n));
designateJettys.add(cloudJettys.get(n));
}
waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, MAX_WAIT_TIME);
cloudClient.getZkStateReader().getZkClient().printLayoutToStdOut();
int numRestarts = 4; // 1 + random().nextInt(5);
boolean sawLiveDesignate = false;
int numRestarts = 1 + random().nextInt(TEST_NIGHTLY ? 12 : 2);
for (int i = 0; i < numRestarts; i++) {
log.info("Rolling restart #{}", i + 1);
for (CloudJettyRunner cloudJetty : overseerDesignates) {
for (CloudJettyRunner cloudJetty : designateJettys) {
log.info("Restarting {}", cloudJetty);
chaosMonkey.stopJetty(cloudJetty);
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()));
fail("No overseer designate as leader found after restart #" + (i + 1) + ": " + leader);
cloudClient.getZkStateReader().updateLiveNodes();
boolean liveDesignates = CollectionUtils.intersection(cloudClient.getZkStateReader().getClusterState().getLiveNodes(), designates).size() > 0;
if (liveDesignates) {
sawLiveDesignate = true;
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()));
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, MAX_WAIT_TIME);
assertTrue("Unable to restart (#" + i + "): " + cloudJetty, ChaosMonkey.start(cloudJetty.jetty));
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()));
fail("No overseer leader found after restart #" + (i + 1) + ": " + leader);
}
cloudClient.getZkStateReader().updateLiveNodes();
sawLiveDesignate = CollectionUtils.intersection(cloudClient.getZkStateReader().getClusterState().getLiveNodes(), designates).size() > 0;
}
}
assertTrue("Test may not be working if we never saw a live designate", sawLiveDesignate);
leader = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
assertNotNull(leader);

View File

@ -567,19 +567,25 @@ public class ChaosMonkey {
jetty.start();
} catch (Exception e) {
jetty.stop();
Thread.sleep(2000);
Thread.sleep(3000);
try {
jetty.start();
} catch (Exception e2) {
jetty.stop();
Thread.sleep(5000);
Thread.sleep(10000);
try {
jetty.start();
} catch (Exception e3) {
log.error("Could not get the port to start jetty again", e3);
// we coud not get the port
jetty.stop();
return false;
Thread.sleep(30000);
try {
jetty.start();
} catch (Exception e4) {
log.error("Could not get the port to start jetty again", e4);
// we coud not get the port
jetty.stop();
return false;
}
}
}
}