You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2017/04/27 23:38:39 UTC
[1/2] lucene-solr:jira/solr-10233: Added ChaosMonkey test without
safe leader for passive replicas
Repository: lucene-solr
Updated Branches:
refs/heads/jira/solr-10233 304add6f6 -> a342edd9e
Added ChaosMonkey test without safe leader for passive replicas
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/2c133d4c
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/2c133d4c
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/2c133d4c
Branch: refs/heads/jira/solr-10233
Commit: 2c133d4cfb533900dcb72784c12b3829e8277c65
Parents: 304add6
Author: Tomas Fernandez Lobbe <tf...@apache.org>
Authored: Thu Apr 27 16:27:46 2017 -0700
Committer: Tomas Fernandez Lobbe <tf...@apache.org>
Committed: Thu Apr 27 16:27:46 2017 -0700
----------------------------------------------------------------------
.../processor/DistributedUpdateProcessor.java | 5 +
...keyNothingIsSafeWithPassiveReplicasTest.java | 431 +++++++++++++++++++
...MonkeySafeLeaderWithPassiveReplicasTest.java | 8 +
.../java/org/apache/solr/cloud/ChaosMonkey.java | 85 +++-
4 files changed, 517 insertions(+), 12 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2c133d4c/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index e9f63d5..41fde18 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -1885,6 +1885,11 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
nodes = getCollectionUrls(req, req.getCore().getCoreDescriptor()
.getCloudDescriptor().getCollectionName(), EnumSet.of(Replica.Type.APPEND,Replica.Type.REALTIME));
+ if (nodes == null) {
+ // This could happen if there are only passive replicas
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+ "Unable to distribute commit operation. No replicas available of types " + Replica.Type.APPEND + " or " + Replica.Type.REALTIME);
+ }
if (isLeader && nodes.size() == 1) {
singleLeader = true;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2c133d4c/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPassiveReplicasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPassiveReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPassiveReplicasTest.java
new file mode 100644
index 0000000..9daec5b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeWithPassiveReplicasTest.java
@@ -0,0 +1,431 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud;
+
+import java.lang.invoke.MethodHandles;
+import java.net.ConnectException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.http.client.HttpClient;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.SolrTestCaseJ4.SuppressObjectReleaseTracker;
+import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.IOUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Slow
+@SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776")
+//@ThreadLeakLingering(linger = 60000)
+@SuppressObjectReleaseTracker(bugUrl="Testing purposes")
+public class ChaosMonkeyNothingIsSafeWithPassiveReplicasTest extends AbstractFullDistribZkTestBase {
+ private static final int FAIL_TOLERANCE = 100;
+
+ private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+ private static final Integer RUN_LENGTH = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.runlength", "-1"));
+
+ private final boolean useAppendReplicas = random().nextBoolean();
+
+ private final int numPassiveReplicas;
+ private final int numRealtimeOrAppendReplicas;
+
+ protected int getPassiveReplicaCount() {
+ return numPassiveReplicas;
+ }
+
+ @BeforeClass
+ public static void beforeSuperClass() {
+ schemaString = "schema15.xml"; // we need a string id
+ System.setProperty("solr.autoCommit.maxTime", "15000");
+ setErrorHook();
+ }
+
+ @AfterClass
+ public static void afterSuperClass() {
+ System.clearProperty("solr.autoCommit.maxTime");
+ clearErrorHook();
+ }
+
+ protected static final String[] fieldNames = new String[]{"f_i", "f_f", "f_d", "f_l", "f_dt"};
+ protected static final RandVal[] randVals = new RandVal[]{rint, rfloat, rdouble, rlong, rdate};
+
+ private int clientSoTimeout;
+
+ public String[] getFieldNames() {
+ return fieldNames;
+ }
+
+ public RandVal[] getRandValues() {
+ return randVals;
+ }
+
+ @Override
+ public void distribSetUp() throws Exception {
+ super.distribSetUp();
+ // can help to hide this when testing and looking at logs
+ //ignoreException("shard update error");
+ useFactory("solr.StandardDirectoryFactory");
+ }
+
+ public ChaosMonkeyNothingIsSafeWithPassiveReplicasTest() {
+ super();
+ numPassiveReplicas = random().nextInt(TEST_NIGHTLY ? 3 : 2) + 1;;
+ numRealtimeOrAppendReplicas = random().nextInt(TEST_NIGHTLY ? 3 : 2) + 1;;
+ sliceCount = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.slicecount", "-1"));
+ if (sliceCount == -1) {
+ sliceCount = random().nextInt(TEST_NIGHTLY ? 3 : 2) + 1;
+ }
+
+ int numNodes = sliceCount * (numRealtimeOrAppendReplicas + numPassiveReplicas);
+ fixShardCount(numNodes);
+ log.info("Starting ChaosMonkey test with {} shards and {} nodes", sliceCount, numNodes);
+
+ // None of the operations used here are particularly costly, so this should work.
+ // Using this low timeout will also help us catch index stalling.
+ clientSoTimeout = 5000;
+ }
+
+ @Override
+ protected boolean useAppendReplicas() {
+ return useAppendReplicas;
+ }
+
+ @Test
+ public void test() throws Exception {
+ cloudClient.setSoTimeout(clientSoTimeout);
+ DocCollection docCollection = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION);
+ assertEquals(this.sliceCount, docCollection.getSlices().size());
+ Slice s = docCollection.getSlice("shard1");
+ assertNotNull(s);
+ assertEquals("Unexpected number of replicas. Collection: " + docCollection, numRealtimeOrAppendReplicas + numPassiveReplicas, s.getReplicas().size());
+ assertEquals("Unexpected number of passive replicas. Collection: " + docCollection, numPassiveReplicas, s.getReplicas(EnumSet.of(Replica.Type.PASSIVE)).size());
+ assertEquals(useAppendReplicas()?0:numRealtimeOrAppendReplicas, s.getReplicas(EnumSet.of(Replica.Type.REALTIME)).size());
+ assertEquals(useAppendReplicas()?numRealtimeOrAppendReplicas:0, s.getReplicas(EnumSet.of(Replica.Type.APPEND)).size());
+
+ boolean testSuccessful = false;
+ try {
+ handle.clear();
+ handle.put("timestamp", SKIPVAL);
+ ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+ // make sure we have leaders for each shard
+ for (int j = 1; j < sliceCount; j++) {
+ zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+ } // make sure we again have leaders for each shard
+
+ waitForRecoveriesToFinish(false);
+
+ // we cannot do delete by query
+ // as it's not supported for recovery
+ del("*:*");
+
+ List<StoppableThread> threads = new ArrayList<>();
+ List<StoppableIndexingThread> indexTreads = new ArrayList<>();
+ int threadCount = TEST_NIGHTLY ? 3 : 1;
+ int i = 0;
+ for (i = 0; i < threadCount; i++) {
+ StoppableIndexingThread indexThread = new StoppableIndexingThread(controlClient, cloudClient, Integer.toString(i), true);
+ threads.add(indexThread);
+ indexTreads.add(indexThread);
+ indexThread.start();
+ }
+
+ threadCount = 1;
+ i = 0;
+ for (i = 0; i < threadCount; i++) {
+ StoppableSearchThread searchThread = new StoppableSearchThread(cloudClient);
+ threads.add(searchThread);
+ searchThread.start();
+ }
+
+ // TODO: we only do this sometimes so that we can sometimes compare against control,
+ // it's currently hard to know what requests failed when using ConcurrentSolrUpdateServer
+ boolean runFullThrottle = random().nextBoolean();
+ if (runFullThrottle) {
+ FullThrottleStoppableIndexingThread ftIndexThread = new FullThrottleStoppableIndexingThread(
+ clients, "ft1", true);
+ threads.add(ftIndexThread);
+ ftIndexThread.start();
+ }
+
+ chaosMonkey.startTheMonkey(true, 10000);
+ try {
+ long runLength;
+ if (RUN_LENGTH != -1) {
+ runLength = RUN_LENGTH;
+ } else {
+ int[] runTimes;
+ if (TEST_NIGHTLY) {
+ runTimes = new int[] {5000, 6000, 10000, 15000, 25000, 30000,
+ 30000, 45000, 90000, 120000};
+ } else {
+ runTimes = new int[] {5000, 7000, 15000};
+ }
+ runLength = runTimes[random().nextInt(runTimes.length - 1)];
+ }
+
+ Thread.sleep(runLength);
+ } finally {
+ chaosMonkey.stopTheMonkey();
+ }
+
+ // ideally this should go into chaosMonkey
+ restartZk(1000 * (5 + random().nextInt(4)));
+
+ for (StoppableThread indexThread : threads) {
+ indexThread.safeStop();
+ }
+
+ // start any downed jetties to be sure we still will end up with a leader per shard...
+
+ // wait for stop...
+ for (StoppableThread indexThread : threads) {
+ indexThread.join();
+ }
+
+ // try and wait for any replications and what not to finish...
+
+ Thread.sleep(2000);
+
+ // wait until there are no recoveries...
+ waitForThingsToLevelOut(Integer.MAX_VALUE);//Math.round((runLength / 1000.0f / 3.0f)));
+
+ // make sure we again have leaders for each shard
+ for (int j = 1; j < sliceCount; j++) {
+ zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 30000);
+ }
+
+ commit();
+
+ // TODO: assert we didnt kill everyone
+
+ zkStateReader.updateLiveNodes();
+ assertTrue(zkStateReader.getClusterState().getLiveNodes().size() > 0);
+
+
+ // we expect full throttle fails, but cloud client should not easily fail
+ for (StoppableThread indexThread : threads) {
+ if (indexThread instanceof StoppableIndexingThread && !(indexThread instanceof FullThrottleStoppableIndexingThread)) {
+ int failCount = ((StoppableIndexingThread) indexThread).getFailCount();
+ assertFalse("There were too many update fails (" + failCount + " > " + FAIL_TOLERANCE
+ + ") - we expect it can happen, but shouldn't easily", failCount > FAIL_TOLERANCE);
+ }
+ }
+
+
+ Set<String> addFails = getAddFails(indexTreads);
+ Set<String> deleteFails = getDeleteFails(indexTreads);
+ // full throttle thread can
+ // have request fails
+ checkShardConsistency(!runFullThrottle, true, addFails, deleteFails);
+
+ long ctrlDocs = controlClient.query(new SolrQuery("*:*")).getResults()
+ .getNumFound();
+
+ // ensure we have added more than 0 docs
+ long cloudClientDocs = cloudClient.query(new SolrQuery("*:*"))
+ .getResults().getNumFound();
+
+ assertTrue("Found " + ctrlDocs + " control docs", cloudClientDocs > 0);
+
+ if (VERBOSE) System.out.println("control docs:"
+ + controlClient.query(new SolrQuery("*:*")).getResults()
+ .getNumFound() + "\n\n");
+
+ // try and make a collection to make sure the overseer has survived the expiration and session loss
+
+ // sometimes we restart zookeeper as well
+ if (random().nextBoolean()) {
+ restartZk(1000 * (5 + random().nextInt(4)));
+ }
+
+ try (CloudSolrClient client = createCloudClient("collection1")) {
+ createCollection(null, "testcollection",
+ 1, 1, 1, client, null, "conf1");
+
+ }
+ List<Integer> numShardsNumReplicas = new ArrayList<>(2);
+ numShardsNumReplicas.add(1);
+ numShardsNumReplicas.add(1 + getPassiveReplicaCount());
+ checkForCollection("testcollection", numShardsNumReplicas, null);
+
+ testSuccessful = true;
+ } finally {
+ if (!testSuccessful) {
+ printLayout();
+ }
+ }
+ }
+
+ private Set<String> getAddFails(List<StoppableIndexingThread> threads) {
+ Set<String> addFails = new HashSet<String>();
+ for (StoppableIndexingThread thread : threads) {
+ addFails.addAll(thread.getAddFails());
+ }
+ return addFails;
+ }
+
+ private Set<String> getDeleteFails(List<StoppableIndexingThread> threads) {
+ Set<String> deleteFails = new HashSet<String>();
+ for (StoppableIndexingThread thread : threads) {
+ deleteFails.addAll(thread.getDeleteFails());
+ }
+ return deleteFails;
+ }
+
+ class FullThrottleStoppableIndexingThread extends StoppableIndexingThread {
+ private CloseableHttpClient httpClient = HttpClientUtil.createClient(null);
+ private volatile boolean stop = false;
+ int clientIndex = 0;
+ private ConcurrentUpdateSolrClient cusc;
+ private List<SolrClient> clients;
+ private AtomicInteger fails = new AtomicInteger();
+
+ public FullThrottleStoppableIndexingThread(List<SolrClient> clients,
+ String id, boolean doDeletes) {
+ super(controlClient, cloudClient, id, doDeletes);
+ setName("FullThrottleStopableIndexingThread");
+ setDaemon(true);
+ this.clients = clients;
+
+ cusc = new ErrorLoggingConcurrentUpdateSolrClient(((HttpSolrClient) clients.get(0)).getBaseURL(), httpClient, 8, 2);
+ cusc.setConnectionTimeout(10000);
+ cusc.setSoTimeout(clientSoTimeout);
+ }
+
+ @Override
+ public void run() {
+ int i = 0;
+ int numDeletes = 0;
+ int numAdds = 0;
+
+ while (true && !stop) {
+ String id = this.id + "-" + i;
+ ++i;
+
+ if (doDeletes && random().nextBoolean() && deletes.size() > 0) {
+ String delete = deletes.remove(0);
+ try {
+ numDeletes++;
+ cusc.deleteById(delete);
+ } catch (Exception e) {
+ changeUrlOnError(e);
+ fails.incrementAndGet();
+ }
+ }
+
+ try {
+ numAdds++;
+ if (numAdds > (TEST_NIGHTLY ? 4002 : 197))
+ continue;
+ SolrInputDocument doc = getDoc(
+ "id",
+ id,
+ i1,
+ 50,
+ t1,
+ "Saxon heptarchies that used to rip around so in old times and raise Cain. My, you ought to seen old Henry the Eight when he was in bloom. He WAS a blossom. He used to marry a new wife every day, and chop off her head next morning. And he would do it just as indifferent as if ");
+ cusc.add(doc);
+ } catch (Exception e) {
+ changeUrlOnError(e);
+ fails.incrementAndGet();
+ }
+
+ if (doDeletes && random().nextBoolean()) {
+ deletes.add(id);
+ }
+
+ }
+
+ log.info("FT added docs:" + numAdds + " with " + fails + " fails" + " deletes:" + numDeletes);
+ }
+
+ private void changeUrlOnError(Exception e) {
+ if (e instanceof ConnectException) {
+ clientIndex++;
+ if (clientIndex > clients.size() - 1) {
+ clientIndex = 0;
+ }
+ cusc.shutdownNow();
+ cusc = new ErrorLoggingConcurrentUpdateSolrClient(((HttpSolrClient) clients.get(clientIndex)).getBaseURL(),
+ httpClient, 30, 3);
+ }
+ }
+
+ @Override
+ public void safeStop() {
+ stop = true;
+ cusc.blockUntilFinished();
+ cusc.shutdownNow();
+ IOUtils.closeQuietly(httpClient);
+ }
+
+ @Override
+ public int getFailCount() {
+ return fails.get();
+ }
+
+ @Override
+ public Set<String> getAddFails() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Set<String> getDeleteFails() {
+ throw new UnsupportedOperationException();
+ }
+
+ };
+
+
+ // skip the randoms - they can deadlock...
+ @Override
+ protected void indexr(Object... fields) throws Exception {
+ SolrInputDocument doc = getDoc(fields);
+ indexDoc(doc);
+ }
+
+ static class ErrorLoggingConcurrentUpdateSolrClient extends ConcurrentUpdateSolrClient {
+ public ErrorLoggingConcurrentUpdateSolrClient(String serverUrl, HttpClient httpClient, int queueSize, int threadCount) {
+ super(serverUrl, httpClient, queueSize, threadCount, null, false);
+ }
+ @Override
+ public void handleError(Throwable ex) {
+ log.warn("cusc error", ex);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2c133d4c/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPassiveReplicasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPassiveReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPassiveReplicasTest.java
index 5fee604..46dc837 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPassiveReplicasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderWithPassiveReplicasTest.java
@@ -42,12 +42,20 @@ public class ChaosMonkeySafeLeaderWithPassiveReplicasTest extends AbstractFullDi
private static final Integer RUN_LENGTH = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.runlength", "-1"));
+ private final boolean useAppendReplicas = random().nextBoolean();
+
private final int numPassiveReplicas;
private final int numRealtimeOrAppendReplicas;
+ @Override
protected int getPassiveReplicaCount() {
return numPassiveReplicas;
}
+
+ @Override
+ protected boolean useAppendReplicas() {
+ return useAppendReplicas;
+ }
@BeforeClass
public static void beforeSuperClass() {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2c133d4c/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
index 2ad42d1..fd2d15b 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
@@ -29,7 +29,9 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.cloud.AbstractFullDistribZkTestBase.CloudJettyRunner;
+import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Replica.Type;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
@@ -369,16 +371,32 @@ public class ChaosMonkey {
return null;
}
+ boolean canKillIndexer = canKillIndexer(slice);
+
+ if (!canKillIndexer) {
+ monkeyLog("Number of indexer nodes (realtime or append) is not enough to kill one of them, Will only choose a passive replica to kill");
+ }
+
int chance = chaosRandom.nextInt(10);
- CloudJettyRunner cjetty;
- if (chance <= 5 && aggressivelyKillLeaders) {
+ CloudJettyRunner cjetty = null;
+ if (chance <= 5 && aggressivelyKillLeaders && canKillIndexer) {
// if killLeader, really aggressively go after leaders
cjetty = shardToLeaderJetty.get(slice);
} else {
- // get random shard
List<CloudJettyRunner> jetties = shardToJetty.get(slice);
- int index = chaosRandom.nextInt(jetties.size());
- cjetty = jetties.get(index);
+ // get random node
+ int attempt = 0;
+ while (true) {
+ attempt++;
+ int index = chaosRandom.nextInt(jetties.size());
+ cjetty = jetties.get(index);
+ if (canKillIndexer || getTypeForJetty(slice, cjetty) == Replica.Type.PASSIVE) {
+ break;
+ } else if (attempt > 20) {
+ monkeyLog("Can't kill indexer nodes (realtime or append) and couldn't find a random passive node after 20 attempts - monkey cannot kill :(");
+ return null;
+ }
+ }
ZkNodeProps leader = null;
try {
@@ -403,7 +421,7 @@ public class ChaosMonkey {
return null;
}
- boolean isLeader = leader.getStr(ZkStateReader.NODE_NAME_PROP).equals(jetties.get(index).nodeName)
+ boolean isLeader = leader.getStr(ZkStateReader.NODE_NAME_PROP).equals(cjetty.nodeName)
|| rtIsLeader;
if (!aggressivelyKillLeaders && isLeader) {
// we don't kill leaders...
@@ -424,18 +442,61 @@ public class ChaosMonkey {
return cjetty;
}
- private int checkIfKillIsLegal(String slice, int numActive) throws KeeperException, InterruptedException {
- for (CloudJettyRunner cloudJetty : shardToJetty.get(slice)) {
+ private Type getTypeForJetty(String sliceName, CloudJettyRunner cjetty) {
+ DocCollection docCollection = zkStateReader.getClusterState().getCollection(collection);
+
+ Slice slice = docCollection.getSlice(sliceName);
+
+ ZkNodeProps props = slice.getReplicasMap().get(cjetty.coreNodeName);
+ if (props == null) {
+ throw new RuntimeException("shard name " + cjetty.coreNodeName + " not found in " + slice.getReplicasMap().keySet());
+ }
+ return Replica.Type.valueOf(props.getStr(ZkStateReader.REPLICA_TYPE));
+ }
+
+ private boolean canKillIndexer(String sliceName) throws KeeperException, InterruptedException {
+ int numIndexersFoundInShard = 0;
+ for (CloudJettyRunner cloudJetty : shardToJetty.get(sliceName)) {
// get latest cloud state
zkStateReader.forceUpdateCollection(collection);
- Slice theShards = zkStateReader.getClusterState().getSlicesMap(collection)
- .get(slice);
+ DocCollection docCollection = zkStateReader.getClusterState().getCollection(collection);
+
+ Slice slice = docCollection.getSlice(sliceName);
+
+ ZkNodeProps props = slice.getReplicasMap().get(cloudJetty.coreNodeName);
+ if (props == null) {
+ throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + slice.getReplicasMap().keySet());
+ }
+
+ final Replica.State state = Replica.State.getState(props.getStr(ZkStateReader.STATE_PROP));
+ final Replica.Type replicaType = Replica.Type.valueOf(props.getStr(ZkStateReader.REPLICA_TYPE));
+ final String nodeName = props.getStr(ZkStateReader.NODE_NAME_PROP);
+
+ if (cloudJetty.jetty.isRunning()
+ && state == Replica.State.ACTIVE
+ && (replicaType == Replica.Type.APPEND || replicaType == Replica.Type.REALTIME)
+ && zkStateReader.getClusterState().liveNodesContain(nodeName)) {
+ numIndexersFoundInShard++;
+ }
+ }
+ return numIndexersFoundInShard > 1;
+ }
+
+ private int checkIfKillIsLegal(String sliceName, int numActive) throws KeeperException, InterruptedException {
+ for (CloudJettyRunner cloudJetty : shardToJetty.get(sliceName)) {
+
+ // get latest cloud state
+ zkStateReader.forceUpdateCollection(collection);
+
+ DocCollection docCollection = zkStateReader.getClusterState().getCollection(collection);
+
+ Slice slice = docCollection.getSlice(sliceName);
- ZkNodeProps props = theShards.getReplicasMap().get(cloudJetty.coreNodeName);
+ ZkNodeProps props = slice.getReplicasMap().get(cloudJetty.coreNodeName);
if (props == null) {
- throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + theShards.getReplicasMap().keySet());
+ throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + slice.getReplicasMap().keySet());
}
final Replica.State state = Replica.State.getState(props.getStr(ZkStateReader.STATE_PROP));
[2/2] lucene-solr:jira/solr-10233: Fix ChaosMonkey expire connection
and connection loss properties
Posted by tf...@apache.org.
Fix ChaosMonkey expire connection and connection loss properties
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/a342edd9
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/a342edd9
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/a342edd9
Branch: refs/heads/jira/solr-10233
Commit: a342edd9eee95c30eabd00824a7c69f1d36ba33a
Parents: 2c133d4
Author: Tomas Fernandez Lobbe <tf...@apache.org>
Authored: Thu Apr 27 16:38:24 2017 -0700
Committer: Tomas Fernandez Lobbe <tf...@apache.org>
Committed: Thu Apr 27 16:38:24 2017 -0700
----------------------------------------------------------------------
.../src/java/org/apache/solr/cloud/ChaosMonkey.java | 8 ++++----
1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a342edd9/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
index fd2d15b..f42831b 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
@@ -63,8 +63,8 @@ public class ChaosMonkey {
private Map<String,List<CloudJettyRunner>> shardToJetty;
private static final Boolean MONKEY_ENABLED = Boolean.valueOf(System.getProperty("solr.tests.cloud.cm.enabled", "true"));
- private static final Boolean CONN_LOSS = Boolean.valueOf(System.getProperty("solr.tests.cloud.cm.connloss", null));
- private static final Boolean EXP = Boolean.valueOf(System.getProperty("solr.tests.cloud.cm.exp", null));
+ private static final String CONN_LOSS = System.getProperty("solr.tests.cloud.cm.connloss");
+ private static final String EXP = System.getProperty("solr.tests.cloud.cm.exp");
private ZkTestServer zkServer;
private ZkStateReader zkStateReader;
@@ -108,12 +108,12 @@ public class ChaosMonkey {
}
if (EXP != null) {
- expireSessions = EXP;
+ expireSessions = Boolean.parseBoolean(EXP);
} else {
expireSessions = chaosRandom.nextBoolean();
}
if (CONN_LOSS != null) {
- causeConnectionLoss = CONN_LOSS;
+ causeConnectionLoss = Boolean.parseBoolean(CONN_LOSS);
} else {
causeConnectionLoss = chaosRandom.nextBoolean();
}