mirror of https://github.com/apache/lucene.git
SOLR-11451: ComputePlanActionTest.testNodeLost() failure
This commit is contained in:
parent
a28474ec6f
commit
4f52d2db10
|
@ -101,12 +101,12 @@ public class ComputePlanAction extends TriggerActionBase {
|
|||
case NODEADDED:
|
||||
suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
|
||||
.hint(Policy.Suggester.Hint.TARGET_NODE, event.getProperty(TriggerEvent.NODE_NAMES));
|
||||
log.debug("Created suggester with targetNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
|
||||
log.debug("NODEADDED Created suggester with targetNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
|
||||
break;
|
||||
case NODELOST:
|
||||
suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
|
||||
.hint(Policy.Suggester.Hint.SRC_NODE, event.getProperty(TriggerEvent.NODE_NAMES));
|
||||
log.debug("Created suggester with srcNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
|
||||
log.debug("NODELOST Created suggester with srcNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
|
||||
break;
|
||||
default:
|
||||
throw new UnsupportedOperationException("No support for events other than nodeAdded and nodeLost, received: " + event.getEventType());
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.Map;
|
|||
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
|
||||
import org.apache.solr.common.MapWriter;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
import org.apache.solr.util.IdUtils;
|
||||
|
||||
/**
|
||||
|
@ -145,11 +146,6 @@ public class TriggerEvent implements MapWriter {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.getClass().getSimpleName() + "{" +
|
||||
"id='" + id + '\'' +
|
||||
", source='" + source + '\'' +
|
||||
", eventTime=" + eventTime +
|
||||
", properties=" + properties +
|
||||
'}';
|
||||
return Utils.toJSONString(this);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.solr.cloud.autoscaling;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
@ -55,7 +56,7 @@ import static org.apache.solr.common.params.CollectionParams.CollectionAction.MO
|
|||
/**
|
||||
* Test for {@link ComputePlanAction}
|
||||
*/
|
||||
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.cloud.Overseer=DEBUG;org.apache.solr.cloud.overseer=DEBUG;")
|
||||
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.cloud.Overseer=DEBUG;org.apache.solr.cloud.overseer=DEBUG;org.apache.solr.client.solrj.impl.SolrClientDataProvider=DEBUG;")
|
||||
public class ComputePlanActionTest extends SolrCloudTestCase {
|
||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||
|
||||
|
@ -143,13 +144,14 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
|
|||
// let's start a node so that we have at least two
|
||||
JettySolrRunner runner = cluster.startJettySolrRunner();
|
||||
String node = runner.getNodeName();
|
||||
AssertingTriggerAction.expectedNode = node;
|
||||
|
||||
CloudSolrClient solrClient = cluster.getSolrClient();
|
||||
String setTriggerCommand = "{" +
|
||||
"'set-trigger' : {" +
|
||||
"'name' : 'node_lost_trigger'," +
|
||||
"'event' : 'nodeLost'," +
|
||||
"'waitFor' : '1s'," +
|
||||
"'waitFor' : '7s'," +
|
||||
"'enabled' : true," +
|
||||
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
|
||||
"{'name':'test','class':'" + ComputePlanActionTest.AssertingTriggerAction.class.getName() + "'}]" +
|
||||
|
@ -186,14 +188,15 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
|
|||
break;
|
||||
}
|
||||
}
|
||||
log.info("$$$$$$$$$$$$$$Stopped_node : {}", node);
|
||||
cluster.waitForAllNodes(30);
|
||||
|
||||
assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(5, TimeUnit.SECONDS));
|
||||
assertTrue("Trigger was not fired even after 10 seconds", triggerFiredLatch.await(10, TimeUnit.SECONDS));
|
||||
assertTrue(fired.get());
|
||||
Map context = actionContextPropsRef.get();
|
||||
assertNotNull(context);
|
||||
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
|
||||
assertNotNull("The operations computed by ComputePlanAction should not be null , "+ getDataProviderState(), operations);
|
||||
assertNotNull("The operations computed by ComputePlanAction should not be null , " + getDataProviderState() + eventRef.get(), operations);
|
||||
assertEquals("ComputePlanAction should have computed exactly 1 operation", 1, operations.size());
|
||||
SolrRequest solrRequest = operations.get(0);
|
||||
SolrParams params = solrRequest.getParams();
|
||||
|
@ -214,13 +217,16 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
|
|||
String result = "SolrClientDataProvider.DEBUG";
|
||||
if(SolrClientDataProvider.INST != null) {
|
||||
result+= Utils.toJSONString(SolrClientDataProvider.INST);
|
||||
if(SolrClientDataProvider.INST.config != null) result+= Utils.toJSONString(SolrClientDataProvider.INST.config);
|
||||
if (SolrClientDataProvider.INST.config != null)
|
||||
result += "autoscalingconf: " + Utils.toJSONString(SolrClientDataProvider.INST.config);
|
||||
}
|
||||
return result;
|
||||
|
||||
}
|
||||
|
||||
public void testNodeWithMultipleReplicasLost() throws Exception {
|
||||
AssertingTriggerAction.expectedNode = null;
|
||||
|
||||
// start 3 more nodes
|
||||
cluster.startJettySolrRunner();
|
||||
cluster.startJettySolrRunner();
|
||||
|
@ -280,7 +286,7 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
|
|||
Map context = actionContextPropsRef.get();
|
||||
assertNotNull(context);
|
||||
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
|
||||
assertNotNull("The operations computed by ComputePlanAction should not be null "+getDataProviderState(), operations);
|
||||
assertNotNull("The operations computed by ComputePlanAction should not be null " + getDataProviderState() + actionContextPropsRef.get(), operations);
|
||||
operations.forEach(solrRequest -> log.info(solrRequest.getParams().toString()));
|
||||
assertEquals("ComputePlanAction should have computed exactly 2 operation", 2, operations.size());
|
||||
|
||||
|
@ -294,6 +300,7 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
|
|||
|
||||
@Test
|
||||
public void testNodeAdded() throws Exception {
|
||||
AssertingTriggerAction.expectedNode = null;
|
||||
CloudSolrClient solrClient = cluster.getSolrClient();
|
||||
String setTriggerCommand = "{" +
|
||||
"'set-trigger' : {" +
|
||||
|
@ -346,7 +353,7 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
|
|||
Map context = actionContextPropsRef.get();
|
||||
assertNotNull(context);
|
||||
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
|
||||
assertNotNull("The operations computed by ComputePlanAction should not be null" + getDataProviderState(), operations);
|
||||
assertNotNull("The operations computed by ComputePlanAction should not be null" + getDataProviderState() + context, operations);
|
||||
assertEquals("ComputePlanAction should have computed exactly 1 operation", 1, operations.size());
|
||||
SolrRequest request = operations.get(0);
|
||||
SolrParams params = request.getParams();
|
||||
|
@ -356,6 +363,7 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
|
|||
}
|
||||
|
||||
public static class AssertingTriggerAction implements TriggerAction {
|
||||
static String expectedNode;
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
|
@ -364,6 +372,10 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
|
|||
|
||||
@Override
|
||||
public void process(TriggerEvent event, ActionContext context) {
|
||||
if (expectedNode != null) {
|
||||
Collection nodes = (Collection) event.getProperty(TriggerEvent.NODE_NAMES);
|
||||
if (nodes == null || !nodes.contains(expectedNode)) return;//this is not the event we are looking for
|
||||
}
|
||||
if (fired.compareAndSet(false, true)) {
|
||||
eventRef.set(event);
|
||||
actionContextPropsRef.set(context.getProperties());
|
||||
|
|
Loading…
Reference in New Issue