You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/12/07 19:22:08 UTC
lucene-solr:jira/solr-11285-sim: Fix problems after merge. Add
support for flaky nodes.
Repository: lucene-solr
Updated Branches:
refs/heads/jira/solr-11285-sim 071d14384 -> e4fed2145
Fix problems after merge. Add support for flaky nodes.
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/e4fed214
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/e4fed214
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/e4fed214
Branch: refs/heads/jira/solr-11285-sim
Commit: e4fed214565809bfcb6269d6f8d31df7d54a4857
Parents: 071d143
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Thu Dec 7 20:21:32 2017 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Thu Dec 7 20:21:32 2017 +0100
----------------------------------------------------------------------
.../cloud/autoscaling/ScheduledTriggers.java | 2 +-
.../cloud/autoscaling/sim/SimCloudManager.java | 35 ++++++++++++++--
.../sim/SimClusterStateProvider.java | 43 ++++++++++++++++----
.../autoscaling/sim/TestComputePlanAction.java | 6 +--
.../autoscaling/sim/TestExecutePlanAction.java | 2 +-
.../cloud/autoscaling/sim/TestLargeCluster.java | 23 ++++++++++-
.../autoscaling/sim/TestTriggerIntegration.java | 6 +--
7 files changed, 97 insertions(+), 20 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4fed214/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
index 2f5c517..0f81300 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
@@ -203,7 +203,7 @@ public class ScheduledTriggers implements Closeable {
this.autoScalingConfig = autoScalingConfig;
// reset cooldown and actionThrottle
- cooldownStart.set(System.nanoTime() - cooldownPeriod.get());
+ cooldownStart.set(cloudManager.getTimeSource().getTime() - cooldownPeriod.get());
actionThrottle.get().reset();
listeners.setAutoScalingConfig(autoScalingConfig);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4fed214/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
index ecfc8f7..9c8cc29 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
@@ -31,6 +31,7 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest;
@@ -98,6 +99,7 @@ public class SimCloudManager implements SolrCloudManager {
private final List<SolrInputDocument> systemColl = Collections.synchronizedList(new ArrayList<>());
private final ExecutorService simCloudManagerPool;
+ private final Map<String, AtomicLong> opCounts = new ConcurrentHashMap<>();
private Overseer.OverseerThread triggerThread;
@@ -227,24 +229,32 @@ public class SimCloudManager implements SolrCloudManager {
/**
* Remove a node from the cluster. This simulates a node lost scenario.
* @param nodeId node id
+ * @param withValues when true, remove also simulated node values. If false
+ * then node values are retained to later simulate
+ * a node that comes back up
*/
- public void simRemoveNode(String nodeId) throws Exception {
+ public void simRemoveNode(String nodeId, boolean withValues) throws Exception {
clusterStateProvider.simRemoveNode(nodeId);
- nodeStateProvider.simRemoveNodeValues(nodeId);
+ if (withValues) {
+ nodeStateProvider.simRemoveNodeValues(nodeId);
+ }
LOG.trace("-- removed node " + nodeId);
}
/**
* Remove a number of randomly selected nodes
* @param number number of nodes to remove
+ * @param withValues when true, remove also simulated node values. If false
+ * then node values are retained to later simulate
+ * a node that comes back up
* @param random random
*/
- public void simRemoveRandomNodes(int number, Random random) throws Exception {
+ public void simRemoveRandomNodes(int number, boolean withValues, Random random) throws Exception {
List<String> nodes = new ArrayList<>(liveNodes);
Collections.shuffle(nodes, random);
int count = Math.min(number, nodes.size());
for (int i = 0; i < count; i++) {
- simRemoveNode(nodes.get(i));
+ simRemoveNode(nodes.get(i), withValues);
}
}
@@ -309,6 +319,15 @@ public class SimCloudManager implements SolrCloudManager {
return stateManager;
}
+ public Map<String, AtomicLong> simGetOpCounts() {
+ return opCounts;
+ }
+
+ public long simGetOpCount(String op) {
+ AtomicLong count = opCounts.get(op);
+ return count != null ? count.get() : 0L;
+ }
+
// --------- interface methods -----------
@@ -360,6 +379,11 @@ public class SimCloudManager implements SolrCloudManager {
}
}
+ private void incrementCount(String op) {
+ AtomicLong count = opCounts.computeIfAbsent(op, o -> new AtomicLong());
+ count.incrementAndGet();
+ }
+
/**
* Handler for autoscaling requests. NOTE: only a specific subset of autoscaling requests is
* supported!
@@ -374,6 +398,7 @@ public class SimCloudManager implements SolrCloudManager {
(req.getParams() != null ? " " + req.getParams().toQueryString() : ""));
if (req.getPath() != null && req.getPath().startsWith("/admin/autoscaling") ||
req.getPath().startsWith("/cluster/autoscaling")) {
+ incrementCount("autoscaling");
ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
params.set(CommonParams.PATH, req.getPath());
LocalSolrQueryRequest queryRequest = new LocalSolrQueryRequest(null, params);
@@ -395,6 +420,7 @@ public class SimCloudManager implements SolrCloudManager {
return rsp;
}
if (req instanceof UpdateRequest) {
+ incrementCount("update");
// support only updates to the system collection
UpdateRequest ureq = (UpdateRequest)req;
if (ureq.getCollection() == null || !ureq.getCollection().equals(CollectionAdminParams.SYSTEM_COLL)) {
@@ -422,6 +448,7 @@ public class SimCloudManager implements SolrCloudManager {
LOG.debug("Invoking Collection Action :{} with params {}", action.toLower(), req.getParams().toQueryString());
NamedList results = new NamedList();
rsp.setResponse(results);
+ incrementCount(action.name());
switch (action) {
case REQUESTSTATUS:
// we complete all async ops immediately
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4fed214/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
index d081fee..d51c059 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
@@ -232,13 +232,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
try {
Set<String> collections = new HashSet<>();
// mark every replica on that node as down
- List<ReplicaInfo> replicas = nodeReplicaMap.get(nodeId);
- if (replicas != null) {
- replicas.forEach(r -> {
- r.getVariables().put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
- collections.add(r.getCollection());
- });
- }
+ setReplicaStates(nodeId, Replica.State.DOWN, collections);
boolean res = liveNodes.remove(nodeId);
if (!collections.isEmpty()) {
cloudManager.submit(new LeaderElection(collections, true));
@@ -249,6 +243,41 @@ public class SimClusterStateProvider implements ClusterStateProvider {
}
}
+ // this method needs to be called under a lock
+ private void setReplicaStates(String nodeId, Replica.State state, Set<String> changedCollections) {
+ List<ReplicaInfo> replicas = nodeReplicaMap.get(nodeId);
+ if (replicas != null) {
+ replicas.forEach(r -> {
+ r.getVariables().put(ZkStateReader.STATE_PROP, state.toString());
+ changedCollections.add(r.getCollection());
+ });
+ }
+ }
+
+ public boolean simRestoreNode(String nodeId) throws Exception {
+ liveNodes.add(nodeId);
+ Set<String> collections = new HashSet<>();
+ lock.lock();
+ try {
+ setReplicaStates(nodeId, Replica.State.RECOVERING, collections);
+ } finally {
+ lock.unlock();
+ }
+ cloudManager.getTimeSource().sleep(1000);
+ lock.lock();
+ try {
+ setReplicaStates(nodeId, Replica.State.ACTIVE, collections);
+ } finally {
+ lock.unlock();
+ }
+ if (!collections.isEmpty()) {
+ cloudManager.submit(new LeaderElection(collections, true));
+ return true;
+ } else {
+ return false;
+ }
+ }
+
/**
* Add a new replica. Note that if any details of a replica (node, coreNodeName, SolrCore name, etc)
* are missing they will be filled in using the policy framework.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4fed214/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
index 5ce0af3..89f3f68 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestComputePlanAction.java
@@ -186,7 +186,7 @@ public class TestComputePlanAction extends SimSolrCloudTestCase {
assertTrue(node2 + "is not live yet", cluster.getClusterStateProvider().getClusterState().liveNodesContain(node2) );
// stop the original node
- cluster.simRemoveNode(node);
+ cluster.simRemoveNode(node, false);
log.info("Stopped_node : {}", node);
assertTrue("Trigger was not fired even after 10 seconds", triggerFiredLatch.await(10, TimeUnit.SECONDS));
@@ -203,7 +203,7 @@ public class TestComputePlanAction extends SimSolrCloudTestCase {
assertEquals("Unexpected node in computed operation", replicas.get(0).getName(), replicaToBeMoved);
// shutdown the extra node that we had started
- cluster.simRemoveNode(node2);
+ cluster.simRemoveNode(node2, false);
}
public void testNodeWithMultipleReplicasLost() throws Exception {
@@ -247,7 +247,7 @@ public class TestComputePlanAction extends SimSolrCloudTestCase {
if (replicas != null && replicas.size() == 2) {
stoppedNodeName = node;
replicasToBeMoved = replicas;
- cluster.simRemoveNode(node);
+ cluster.simRemoveNode(node, false);
break;
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4fed214/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
index bbc1317..8c4c287 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestExecutePlanAction.java
@@ -207,7 +207,7 @@ public class TestExecutePlanAction extends SimSolrCloudTestCase {
assertFalse(otherNodes.isEmpty());
String survivor = otherNodes.get(0);
- cluster.simRemoveNode(sourceNodeName);
+ cluster.simRemoveNode(sourceNodeName, false);
waitForState("Timed out waiting for replicas of collection to be 2 again",
collectionName, clusterShape(1, 2));
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4fed214/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
index f1f0d01..7caac42 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestLargeCluster.java
@@ -44,6 +44,7 @@ import org.apache.solr.cloud.autoscaling.TriggerEvent;
import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
import org.apache.solr.cloud.autoscaling.CapturedEvent;
import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.LogLevel;
@@ -180,11 +181,31 @@ public class TestLargeCluster extends SimSolrCloudTestCase {
int KILL_NODES = 8;
// kill off a number of nodes
for (int i = 0; i < KILL_NODES; i++) {
- cluster.simRemoveNode(nodes.get(i));
+ cluster.simRemoveNode(nodes.get(i), false);
}
log.info("Ready after " + waitForState(collectionName, 90 * KILL_NODES, TimeUnit.SECONDS, clusterShape(2, 15)) + "ms");
+ log.info("OP COUNTS: " + cluster.simGetOpCounts());
+ long moveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
+
+ // simulate a number of flaky nodes
+ int FLAKY_NODES = 10;
+ for (int cnt = 0; cnt < 10; cnt++) {
+ for (int i = KILL_NODES; i < KILL_NODES + FLAKY_NODES; i++) {
+ cluster.simRemoveNode(nodes.get(i), false);
+ }
+ cluster.getTimeSource().sleep(TimeUnit.SECONDS.toMillis(waitForSeconds) * 2);
+ for (int i = KILL_NODES; i < KILL_NODES + FLAKY_NODES; i++) {
+ final String nodeId = nodes.get(i);
+ cluster.submit(() -> cluster.getSimClusterStateProvider().simRestoreNode(nodeId));
+ }
+ }
+
+ log.info("Ready after " + waitForState(collectionName, 30 * nodes.size(), TimeUnit.SECONDS, clusterShape(2, 15)) + "ms");
+ log.info("OP COUNTS: " + cluster.simGetOpCounts());
+ long newMoveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
+ log.info("==== Additional MOVEREPLICA count: " + (newMoveReplicaOps - moveReplicaOps));
}
@Test
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4fed214/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
index 4926e26..179af76 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestTriggerIntegration.java
@@ -213,7 +213,7 @@ public class TestTriggerIntegration extends SimSolrCloudTestCase {
}
// stop the node we had started earlier
- cluster.simRemoveNode(newNode);
+ cluster.simRemoveNode(newNode, false);
if (!triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS)) {
fail("Both triggers should have fired by now");
@@ -287,7 +287,7 @@ public class TestTriggerIntegration extends SimSolrCloudTestCase {
}
assertTrue("The action specified in node_lost_restore_trigger was not instantiated even after 2 seconds", actionInitCalled.getCount() > 0);
- cluster.simRemoveNode(nodeName);
+ cluster.simRemoveNode(nodeName, false);
// ensure that the old trigger sees the stopped node, todo find a better way to do this
timeOut.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
@@ -450,7 +450,7 @@ public class TestTriggerIntegration extends SimSolrCloudTestCase {
}
String lostNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
- cluster.simRemoveNode(lostNodeName);
+ cluster.simRemoveNode(lostNodeName, false);
boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
assertTrue("The trigger did not fire at all", await);
assertTrue(triggerFired.get());