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 2019/01/14 10:40:14 UTC
[lucene-solr] 01/02: SOLR-13072: Wait for autoscaling config
refresh to finish before modifying the cluster and enable the tests for
now.
This is an automated email from the ASF dual-hosted git repository.
ab pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git
commit 229a0894fbcb152db4ca08119da085a002953943
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue Jan 8 16:16:55 2019 +0100
SOLR-13072: Wait for autoscaling config refresh to finish before modifying the cluster
and enable the tests for now.
---
.../cloud/autoscaling/OverseerTriggerThread.java | 13 +++
.../autoscaling/sim/SimClusterStateProvider.java | 4 +
.../autoscaling/sim/TestSimTriggerIntegration.java | 111 +++++++++++++++------
3 files changed, 100 insertions(+), 28 deletions(-)
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
index 41ae59b..de875fb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
@@ -74,6 +74,8 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
private Map<String, AutoScaling.Trigger> activeTriggers = new HashMap<>();
+ private volatile int processedZnodeVersion = -1;
+
private volatile boolean isClosed = false;
private AutoScalingConfig autoScalingConfig;
@@ -109,6 +111,16 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
return scheduledTriggers;
}
+ /**
+ * For tests, to ensure that all processing has been completed in response to an update of /autoscaling.json.
+ * @lucene.internal
+ * @return version of /autoscaling.json for which all configuration updates & processing have been completed.
+ * Until then this value will always be smaller than the current znodeVersion of /autoscaling.json.
+ */
+ public int getProcessedZnodeVersion() {
+ return processedZnodeVersion;
+ }
+
@Override
public boolean isClosed() {
return isClosed;
@@ -248,6 +260,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
log.debug("-- cleaning old nodeLost / nodeAdded markers");
removeMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
removeMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
+ processedZnodeVersion = znodeVersion;
}
}
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 5265594..784c0cb 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
@@ -372,6 +372,10 @@ public class SimClusterStateProvider implements ClusterStateProvider {
}
}
+ public synchronized String simGetOverseerLeader() {
+ return overseerLeader;
+ }
+
// 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);
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
index d7e792e..711b333 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
@@ -35,13 +35,11 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantLock;
-import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
-import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.CloudTestUtils;
import org.apache.solr.cloud.autoscaling.ActionContext;
@@ -147,7 +145,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
}
@Test
- @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 2018-06-18
+ //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
public void testTriggerThrottling() throws Exception {
// for this test we want to create two triggers so we must assert that the actions were created twice
actionInitCalled = new CountDownLatch(2);
@@ -177,6 +175,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
"}}");
+ assertAutoscalingUpdateComplete();
+
// wait until the two instances of action are created
if (!actionInitCalled.await(10000 / SPEED, TimeUnit.MILLISECONDS)) {
fail("Two TriggerAction instances should have been created by now");
@@ -213,6 +213,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"'actions' : [{'name':'test','class':'" + ThrottlingTesterAction.class.getName() + "'}]" +
"}}");
+ assertAutoscalingUpdateComplete();
+
// wait until the two instances of action are created
if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS)) {
fail("Two TriggerAction instances should have been created by now");
@@ -285,6 +287,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
actionInitCalled.await(60, TimeUnit.SECONDS));
+
+ assertAutoscalingUpdateComplete();
// start a new node that we can kill later
final String nodeName = cluster.simAddNode();
@@ -371,7 +375,9 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"'enabled' : true," +
"'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
"}}");
-
+
+ assertAutoscalingUpdateComplete();
+
assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
actionInitCalled.await(60, TimeUnit.SECONDS));
@@ -418,7 +424,9 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"'enabled' : true," +
"'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
"}}");
-
+
+ assertAutoscalingUpdateComplete();
+
assertTrue("Trigger was not init()ed even after await()ing an excessive amount of time",
actionInitCalled.await(60, TimeUnit.SECONDS));
@@ -433,7 +441,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
}
@Test
- @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 2018-06-18
+ //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
public void testNodeAddedTrigger() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
assertAutoScalingRequest
@@ -446,6 +454,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
"}}");
+ assertAutoscalingUpdateComplete();
+
if (!actionInitCalled.await(5000 / SPEED, TimeUnit.MILLISECONDS)) {
fail("The TriggerAction should have been created by now");
}
@@ -474,6 +484,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
"}}");
+ assertAutoscalingUpdateComplete();
+
// this should be a no-op so the action should have been created but init should not be called
if (!actionConstructorCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS)) {
fail("The TriggerAction should have been created by now");
@@ -483,8 +495,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
}
@Test
- // commented 4-Sep-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 26-Mar-2018
- @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
+ //@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
public void testNodeLostTrigger() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
assertAutoScalingRequest
@@ -497,6 +508,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
"}}");
+ assertAutoscalingUpdateComplete();
+
if (!actionInitCalled.await(5000 / SPEED, TimeUnit.MILLISECONDS)) {
fail("The TriggerAction should have been created by now");
}
@@ -525,7 +538,9 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"'enabled' : true," +
"'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
"}}");
-
+
+ assertAutoscalingUpdateComplete();
+
// this should be a no-op so the action should have been created but init should not be called
if (!actionConstructorCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS)) {
fail("The TriggerAction should have been created by now");
@@ -654,7 +669,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
public static long eventQueueActionWait = 5000;
@Test
- @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // this test fails easily
+ //@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072") // this test fails easily
public void testEventQueue() throws Exception {
waitForSeconds = 1;
SolrClient solrClient = cluster.simGetSolrClient();
@@ -670,6 +685,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"'actions' : [{'name':'test','class':'" + TestEventQueueAction.class.getName() + "'}]" +
"}}");
+ assertAutoscalingUpdateComplete();
+
if (!actionInitCalled.await(3000 / SPEED, TimeUnit.MILLISECONDS)) {
fail("The TriggerAction should have been created by now");
}
@@ -692,6 +709,9 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
// kill overseer
cluster.simRestartOverseer(overseerLeader);
cluster.getTimeSource().sleep(5000);
+
+ assertAutoscalingUpdateComplete();
+
// new overseer leader should be elected and run triggers
await = actionInterrupted.await(3000 / SPEED, TimeUnit.MILLISECONDS);
assertTrue("action wasn't interrupted", await);
@@ -707,8 +727,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
}
@Test
- // commented 4-Sep-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") //2018-03-10
- @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 14-Oct-2018
+ // @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
public void testEventFromRestoredState() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
assertAutoScalingRequest
@@ -721,6 +740,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"'actions' : [{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}]" +
"}}");
+ assertAutoscalingUpdateComplete();
+
if (!actionInitCalled.await(10000 / SPEED, TimeUnit.MILLISECONDS)) {
fail("The TriggerAction should have been created by now");
}
@@ -814,7 +835,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
}
@Test
- @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
+ //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
public void testNodeMarkersRegistration() throws Exception {
// for this test we want to create two triggers so we must assert that the actions were created twice
actionInitCalled = new CountDownLatch(2);
@@ -824,8 +845,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
SolrClient solrClient = cluster.simGetSolrClient();
- // pick overseer node
- String overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode();
+ // get overseer node
+ String overseerLeader = cluster.getSimClusterStateProvider().simGetOverseerLeader();
// add a node
String node = cluster.simAddNode();
@@ -842,6 +863,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
// stop overseer
log.info("====== KILL OVERSEER 1");
cluster.simRestartOverseer(overseerLeader);
+ assertAutoscalingUpdateComplete();
+
if (!listener.onChangeLatch.await(10000, TimeUnit.MILLISECONDS)) {
fail("onChange listener didn't execute on cluster change");
}
@@ -892,7 +915,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"'actions' : [{'name':'test','class':'" + TestEventMarkerAction.class.getName() + "'}]" +
"}}");
- overseerLeader = cluster.getSimClusterStateProvider().simGetRandomNode();
+ assertAutoscalingUpdateComplete();
+ overseerLeader = cluster.getSimClusterStateProvider().simGetOverseerLeader();
// create another node
log.info("====== ADD NODE 1");
@@ -906,14 +930,10 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
pathAdded = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + node1;
assertTrue("Path " + pathAdded + " wasn't created", cluster.getDistribStateManager().hasData(pathAdded));
- cluster.getTimeSource().sleep(60000);
- // nodeAdded marker should be consumed now by nodeAdded trigger
- assertFalse("Path " + pathAdded + " should have been deleted",
- cluster.getDistribStateManager().hasData(pathAdded));
-
listener.reset();
events.clear();
- triggerFiredLatch = new CountDownLatch(1);
+ // one nodeAdded (not cleared yet) and one nodeLost
+ triggerFiredLatch = new CountDownLatch(2);
// kill overseer again
log.info("====== KILL OVERSEER 2");
cluster.simRestartOverseer(overseerLeader);
@@ -921,15 +941,32 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
fail("onChange listener didn't execute on cluster change");
}
+ assertAutoscalingUpdateComplete();
if (!triggerFiredLatch.await(120000 / SPEED, TimeUnit.MILLISECONDS)) {
fail("Trigger should have fired by now");
}
- assertEquals(1, events.size());
- TriggerEvent ev = events.iterator().next();
- List<String> nodeNames = (List<String>)ev.getProperty(TriggerEvent.NODE_NAMES);
+ assertEquals(2, events.size());
+ TriggerEvent nodeAdded = null;
+ TriggerEvent nodeLost = null;
+ for (TriggerEvent ev : events) {
+ switch (ev.getEventType()) {
+ case NODEADDED:
+ nodeAdded = ev;
+ break;
+ case NODELOST:
+ nodeLost = ev;
+ break;
+ default:
+ fail("unexpected event type: " + ev);
+ }
+ }
+ assertNotNull("expected nodeAdded event", nodeAdded);
+ assertNotNull("expected nodeLost event", nodeLost);
+ List<String> nodeNames = (List<String>)nodeLost.getProperty(TriggerEvent.NODE_NAMES);
assertTrue(nodeNames.contains(overseerLeader));
- assertEquals(TriggerEventType.NODELOST, ev.getEventType());
+ nodeNames = (List<String>)nodeAdded.getProperty(TriggerEvent.NODE_NAMES);
+ assertTrue(nodeNames.contains(node1));
}
static final Map<String, List<CapturedEvent>> listenerEvents = new ConcurrentHashMap<>();
@@ -1011,6 +1048,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"}" +
"}");
+ assertAutoscalingUpdateComplete();
+
listenerEvents.clear();
failDummyAction = false;
@@ -1121,7 +1160,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
}
@Test
- @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
+ //@BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072")
public void testCooldown() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
failDummyAction = false;
@@ -1149,6 +1188,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"}" +
"}");
+ assertAutoscalingUpdateComplete();
+
listenerCreated = new CountDownLatch(1);
listenerEvents.clear();
@@ -1226,7 +1267,7 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
@Test
- @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // this test is way to sensitive to timing, must be beasted before returned
+ //@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-13072") // this test is way to sensitive to timing, must be beasted before returned
public void testSearchRate() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
String COLL1 = "collection1";
@@ -1265,6 +1306,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
"}" +
"}");
+ assertAutoscalingUpdateComplete();
+
// SolrParams query = params(CommonParams.Q, "*:*");
// for (int i = 0; i < 500; i++) {
// solrClient.query(COLL1, query);
@@ -1380,5 +1423,17 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
((Collection)event.getProperty(TriggerEvent.EVENT_TIMES)).iterator().next());
return event;
}
+
+ private static void assertAutoscalingUpdateComplete() throws Exception {
+ (new TimeOut(30, TimeUnit.SECONDS, cluster.getTimeSource()))
+ .waitFor("OverseerTriggerThread never caught up to the latest znodeVersion", () -> {
+ try {
+ AutoScalingConfig autoscalingConfig = cluster.getDistribStateManager().getAutoScalingConfig();
+ return autoscalingConfig.getZkVersion() == cluster.getOverseerTriggerThread().getProcessedZnodeVersion();
+ } catch (Exception e) {
+ throw new RuntimeException("FAILED", e);
+ }
+ });
+ }
}