You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2014/01/23 00:13:02 UTC
svn commit: r1560557 -
/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
Author: markrmiller
Date: Wed Jan 22 23:13:01 2014
New Revision: 1560557
URL: http://svn.apache.org/r1560557
Log:
tests: try harder to not kill the last replica in a shard
Modified:
lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java?rev=1560557&r1=1560556&r2=1560557&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java Wed Jan 22 23:13:01 2014
@@ -79,6 +79,8 @@ public class ChaosMonkey {
private boolean aggressivelyKillLeaders;
private Map<String,CloudJettyRunner> shardToLeaderJetty;
private volatile long startTime;
+
+ private List<CloudJettyRunner> deadPool = new ArrayList<CloudJettyRunner>();
private Thread monkeyThread;
@@ -319,58 +321,31 @@ public class ChaosMonkey {
public CloudJettyRunner getRandomJetty(String slice, boolean aggressivelyKillLeaders) throws KeeperException, InterruptedException {
-
- int numRunning = 0;
- int numRecovering = 0;
int numActive = 0;
- for (CloudJettyRunner cloudJetty : shardToJetty.get(slice)) {
- 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) {
+ numActive = checkIfKillIsLegal(slice, numActive);
+
+ // TODO: stale state makes this a tough call
+ if (numActive < 2) {
+ // we cannot kill anyone
+ monkeyLog("only one active node in shard - monkey cannot kill :(");
+ return null;
+ }
+
+ // let's check the deadpool count
+ int numRunning = 0;
+ for (CloudJettyRunner cjetty : shardToJetty.get(slice)) {
+ if (!deadPool.contains(cjetty)) {
numRunning++;
}
}
- // TODO: stale state makes this a tough call
- if (numActive < 2) {
+ if (numRunning < 2) {
// we cannot kill anyone
monkeyLog("only one active node in shard - monkey cannot kill :(");
return null;
}
+
Random random = LuceneTestCase.random();
int chance = random.nextInt(10);
CloudJettyRunner cjetty;
@@ -439,6 +414,33 @@ public class ChaosMonkey {
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 {
// get latest cloud state
@@ -473,7 +475,6 @@ public class ChaosMonkey {
stop = false;
monkeyThread = new Thread() {
- private List<CloudJettyRunner> deadPool = new ArrayList<CloudJettyRunner>();
@Override
public void run() {