SOLR-12133: Fix race conditions that caused NodeMarkersRegistrationTest.testNodeMarkersRegistration to fail

This commit is contained in:
Shalin Shekhar Mangar 2018-04-02 21:15:09 +05:30
parent a4789db477
commit 8189e4f972
3 changed files with 35 additions and 13 deletions

View File

@ -142,7 +142,8 @@ Other Changes
* SOLR-12152: Split up TriggerIntegrationTest into multiple tests to isolate and increase reliability. (shalin)
* SOLR-12133: Fix race conditions that caused TriggerIntegrationTest.testEventQueue to fail. (Mark Miller, shalin)
* SOLR-12133: Fix race conditions that caused TriggerIntegrationTest.testEventQueue and testNodeMarkersRegistration
to fail. (Mark Miller, shalin)
* SOLR-12169: Fix ComputePlanActionTest.testSelectedCollections fails on jenkins by aggressively cleaning up
trigger state left by other test methods in the test setup. (shalin)

View File

@ -184,6 +184,7 @@ public class NodeAddedTrigger extends TriggerBase {
private void removeMarker(String nodeName) {
String path = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeName;
try {
log.debug("NodeAddedTrigger {} - removing marker path: {}", name, path);
if (stateManager.hasData(path)) {
stateManager.removeData(path, -1);
}

View File

@ -31,6 +31,7 @@ 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.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
@ -52,9 +53,8 @@ import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAut
public class NodeMarkersRegistrationTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static CountDownLatch actionInitCalled;
private static CountDownLatch triggerFiredLatch;
private static CountDownLatch actionConstructorCalled;
private static CountDownLatch listenerEventLatch;
private static Set<TriggerEvent> events = ConcurrentHashMap.newKeySet();
private static ZkStateReader zkStateReader;
private static ReentrantLock lock = new ReentrantLock();
@ -81,11 +81,8 @@ public class NodeMarkersRegistrationTest extends SolrCloudTestCase {
@Test
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);
// similarly we want both triggers to fire
triggerFiredLatch = new CountDownLatch(2);
actionConstructorCalled = new CountDownLatch(1);
triggerFiredLatch = new CountDownLatch(1);
listenerEventLatch = new CountDownLatch(1);
TestLiveNodesListener listener = registerLiveNodesListener();
NamedList<Object> overSeerStatus = cluster.getSolrClient().request(CollectionAdminRequest.getOverseerStatus());
@ -155,6 +152,18 @@ public class NodeMarkersRegistrationTest extends SolrCloudTestCase {
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String setListener = "{\n" +
" \"set-listener\" : {\n" +
" \"name\" : \"listener_node_added_triggerMR\",\n" +
" \"trigger\" : \"node_added_triggerMR\",\n" +
" \"stage\" : \"STARTED\",\n" +
" \"class\" : \"" + AssertingListener.class.getName() + "\"\n" +
" }\n" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListener);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
overSeerStatus = cluster.getSolrClient().request(CollectionAdminRequest.getOverseerStatus());
overseerLeader = (String) overSeerStatus.get("leader");
overseerLeaderIndex = 0;
@ -178,6 +187,9 @@ public class NodeMarkersRegistrationTest extends SolrCloudTestCase {
pathAdded = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + node1.getNodeName();
assertTrue("Path " + pathAdded + " wasn't created", zkClient().exists(pathAdded, true));
listenerEventLatch.countDown(); // let the trigger thread continue
assertTrue(triggerFiredLatch.await(10, TimeUnit.SECONDS));
Thread.sleep(5000);
// nodeAdded marker should be consumed now by nodeAdded trigger
assertFalse("Path " + pathAdded + " should have been deleted", zkClient().exists(pathAdded, true));
@ -237,10 +249,6 @@ public class NodeMarkersRegistrationTest extends SolrCloudTestCase {
public static class TestEventMarkerAction extends TriggerActionBase {
public TestEventMarkerAction() {
actionConstructorCalled.countDown();
}
@Override
public void process(TriggerEvent event, ActionContext actionContext) {
boolean locked = lock.tryLock();
@ -262,8 +270,20 @@ public class NodeMarkersRegistrationTest extends SolrCloudTestCase {
@Override
public void init(Map<String, String> args) {
log.info("TestEventMarkerAction init");
actionInitCalled.countDown();
super.init(args);
}
}
public static class AssertingListener extends TriggerListenerBase {
@Override
public void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context, Throwable error, String message) throws Exception {
if (!Thread.currentThread().getName().startsWith("ScheduledTrigger")) {
// for future safety
throw new IllegalThreadStateException("AssertingListener should have been invoked by a thread from the scheduled trigger thread pool");
}
log.debug(" --- listener fired for event: {}, stage: {}", event, stage);
listenerEventLatch.await();
log.debug(" --- listener wait complete for event: {}, stage: {}", event, stage);
}
}
}