mirror of https://github.com/apache/lucene.git
tests: try harder to not kill the last replica in a shard
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1560557 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d9e7f46ef9
commit
df7c5e7772
|
@ -80,6 +80,8 @@ public class ChaosMonkey {
|
||||||
private Map<String,CloudJettyRunner> shardToLeaderJetty;
|
private Map<String,CloudJettyRunner> shardToLeaderJetty;
|
||||||
private volatile long startTime;
|
private volatile long startTime;
|
||||||
|
|
||||||
|
private List<CloudJettyRunner> deadPool = new ArrayList<CloudJettyRunner>();
|
||||||
|
|
||||||
private Thread monkeyThread;
|
private Thread monkeyThread;
|
||||||
|
|
||||||
public ChaosMonkey(ZkTestServer zkServer, ZkStateReader zkStateReader,
|
public ChaosMonkey(ZkTestServer zkServer, ZkStateReader zkStateReader,
|
||||||
|
@ -319,51 +321,9 @@ public class ChaosMonkey {
|
||||||
|
|
||||||
public CloudJettyRunner getRandomJetty(String slice, boolean aggressivelyKillLeaders) throws KeeperException, InterruptedException {
|
public CloudJettyRunner getRandomJetty(String slice, boolean aggressivelyKillLeaders) throws KeeperException, InterruptedException {
|
||||||
|
|
||||||
|
|
||||||
int numRunning = 0;
|
|
||||||
int numRecovering = 0;
|
|
||||||
int numActive = 0;
|
int numActive = 0;
|
||||||
|
|
||||||
for (CloudJettyRunner cloudJetty : shardToJetty.get(slice)) {
|
numActive = checkIfKillIsLegal(slice, numActive);
|
||||||
boolean running = true;
|
|
||||||
|
|
||||||
// get latest cloud state
|
|
||||||
zkStateReader.updateClusterState(true);
|
|
||||||
|
|
||||||
Slice theShards = zkStateReader.getClusterState().getSlicesMap(collection)
|
|
||||||
.get(slice);
|
|
||||||
|
|
||||||
ZkNodeProps props = theShards.getReplicasMap().get(cloudJetty.coreNodeName);
|
|
||||||
if (props == null) {
|
|
||||||
throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + theShards.getReplicasMap().keySet());
|
|
||||||
}
|
|
||||||
|
|
||||||
String state = props.getStr(ZkStateReader.STATE_PROP);
|
|
||||||
String nodeName = props.getStr(ZkStateReader.NODE_NAME_PROP);
|
|
||||||
|
|
||||||
|
|
||||||
if (!cloudJetty.jetty.isRunning()
|
|
||||||
|| !state.equals(ZkStateReader.ACTIVE)
|
|
||||||
|| !zkStateReader.getClusterState().liveNodesContain(nodeName)) {
|
|
||||||
running = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cloudJetty.jetty.isRunning()
|
|
||||||
&& state.equals(ZkStateReader.RECOVERING)
|
|
||||||
&& zkStateReader.getClusterState().liveNodesContain(nodeName)) {
|
|
||||||
numRecovering++;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cloudJetty.jetty.isRunning()
|
|
||||||
&& state.equals(ZkStateReader.ACTIVE)
|
|
||||||
&& zkStateReader.getClusterState().liveNodesContain(nodeName)) {
|
|
||||||
numActive++;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (running) {
|
|
||||||
numRunning++;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// TODO: stale state makes this a tough call
|
// TODO: stale state makes this a tough call
|
||||||
if (numActive < 2) {
|
if (numActive < 2) {
|
||||||
|
@ -371,6 +331,21 @@ public class ChaosMonkey {
|
||||||
monkeyLog("only one active node in shard - monkey cannot kill :(");
|
monkeyLog("only one active node in shard - monkey cannot kill :(");
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// let's check the deadpool count
|
||||||
|
int numRunning = 0;
|
||||||
|
for (CloudJettyRunner cjetty : shardToJetty.get(slice)) {
|
||||||
|
if (!deadPool.contains(cjetty)) {
|
||||||
|
numRunning++;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (numRunning < 2) {
|
||||||
|
// we cannot kill anyone
|
||||||
|
monkeyLog("only one active node in shard - monkey cannot kill :(");
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
Random random = LuceneTestCase.random();
|
Random random = LuceneTestCase.random();
|
||||||
int chance = random.nextInt(10);
|
int chance = random.nextInt(10);
|
||||||
CloudJettyRunner cjetty;
|
CloudJettyRunner cjetty;
|
||||||
|
@ -440,6 +415,33 @@ public class ChaosMonkey {
|
||||||
return cjetty;
|
return cjetty;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private int checkIfKillIsLegal(String slice, int numActive)
|
||||||
|
throws KeeperException, InterruptedException {
|
||||||
|
for (CloudJettyRunner cloudJetty : shardToJetty.get(slice)) {
|
||||||
|
|
||||||
|
// get latest cloud state
|
||||||
|
zkStateReader.updateClusterState(true);
|
||||||
|
|
||||||
|
Slice theShards = zkStateReader.getClusterState().getSlicesMap(collection)
|
||||||
|
.get(slice);
|
||||||
|
|
||||||
|
ZkNodeProps props = theShards.getReplicasMap().get(cloudJetty.coreNodeName);
|
||||||
|
if (props == null) {
|
||||||
|
throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + theShards.getReplicasMap().keySet());
|
||||||
|
}
|
||||||
|
|
||||||
|
String state = props.getStr(ZkStateReader.STATE_PROP);
|
||||||
|
String nodeName = props.getStr(ZkStateReader.NODE_NAME_PROP);
|
||||||
|
|
||||||
|
if (cloudJetty.jetty.isRunning()
|
||||||
|
&& state.equals(ZkStateReader.ACTIVE)
|
||||||
|
&& zkStateReader.getClusterState().liveNodesContain(nodeName)) {
|
||||||
|
numActive++;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return numActive;
|
||||||
|
}
|
||||||
|
|
||||||
public SolrServer getRandomClient(String slice) throws KeeperException, InterruptedException {
|
public SolrServer getRandomClient(String slice) throws KeeperException, InterruptedException {
|
||||||
// get latest cloud state
|
// get latest cloud state
|
||||||
zkStateReader.updateClusterState(true);
|
zkStateReader.updateClusterState(true);
|
||||||
|
@ -473,7 +475,6 @@ public class ChaosMonkey {
|
||||||
|
|
||||||
stop = false;
|
stop = false;
|
||||||
monkeyThread = new Thread() {
|
monkeyThread = new Thread() {
|
||||||
private List<CloudJettyRunner> deadPool = new ArrayList<CloudJettyRunner>();
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
|
|
Loading…
Reference in New Issue