You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2017/03/24 08:38:02 UTC
lucene-solr:feature/autoscaling: SOLR-10340: New set-listener and
remove-listener API
Repository: lucene-solr
Updated Branches:
refs/heads/feature/autoscaling e4b3df41a -> 0fb9d1bd1
SOLR-10340: New set-listener and remove-listener API
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/0fb9d1bd
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/0fb9d1bd
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/0fb9d1bd
Branch: refs/heads/feature/autoscaling
Commit: 0fb9d1bd1ba74c60868cde016d1b8aa88875c7a3
Parents: e4b3df4
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Fri Mar 24 14:07:56 2017 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Fri Mar 24 14:07:56 2017 +0530
----------------------------------------------------------------------
solr/CHANGES.txt | 2 +
.../solr/cloud/autoscaling/AutoScaling.java | 43 ++++++
.../cloud/autoscaling/AutoScalingHandler.java | 130 ++++++++++++++++++-
.../resources/apispec/autoscaling.Commands.json | 115 ++++++++++++----
.../autoscaling/AutoScalingHandlerTest.java | 112 +++++++++++++++-
5 files changed, 374 insertions(+), 28 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb9d1bd/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c839a07..3f25c91 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -63,6 +63,8 @@ New Features
* SOLR-10339: New set-trigger and remove-trigger APIs for autoscaling. (shalin)
+* SOLR-10340: New set-listener and remove-listener API for autoscaling. (shalin)
+
Bug Fixes
----------------------
* SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb9d1bd/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
index ae11e78..5ab9110 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScaling.java
@@ -17,6 +17,11 @@
package org.apache.solr.cloud.autoscaling;
+import java.util.Date;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
+
public class AutoScaling {
public enum EventType {
@@ -29,4 +34,42 @@ public class AutoScaling {
INDEXRATE
}
+ public enum TriggerStage {
+ STARTED,
+ ABORTED,
+ SUCCEEDED,
+ FAILED,
+ BEFORE_ACTION,
+ AFTER_ACTION
+ }
+
+ public static interface TriggerListener {
+ public void triggerFired(Trigger trigger, Event event);
+ }
+
+ public static class HttpCallbackListener implements TriggerListener {
+ @Override
+ public void triggerFired(Trigger trigger, Event event) {
+
+ }
+ }
+
+ public static interface Trigger {
+ public String getName();
+
+ public EventType getEventType();
+
+ public boolean isEnabled();
+
+ public Map<String, Object> getProperties();
+ }
+
+ public static interface Event {
+ public String getSource();
+
+ public Date getTime();
+
+ public EventType getType();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb9d1bd/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
index 73508c3..978299e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingHandler.java
@@ -20,10 +20,13 @@ package org.apache.solr.cloud.autoscaling;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
+import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.solr.api.Api;
@@ -88,7 +91,111 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
break;
case "remove-trigger":
handleRemoveTrigger(req, rsp, op);
+ break;
+ case "set-listener":
+ handleSetListener(req, rsp, op);
+ break;
+ case "remove-listener":
+ handleRemoveListener(req, rsp, op);
+ break;
+ }
+ }
+ }
+
+ private void handleRemoveListener(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException {
+ String listenerName = op.getStr("name");
+
+ if (listenerName == null || listenerName.trim().length() == 0) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The listener name cannot be null or empty");
+ }
+ Map<String, Object> autoScalingConf = zkReadAutoScalingConf(container.getZkController().getZkStateReader());
+ Map<String, Object> listeners = (Map<String, Object>) autoScalingConf.get("listeners");
+ if (listeners == null || !listeners.containsKey(listenerName)) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No listener exists with name: " + listenerName);
+ }
+ zkSetListener(container.getZkController().getZkStateReader(), listenerName, null);
+ rsp.getValues().add("result", "success");
+ }
+
+ private void handleSetListener(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException {
+ String listenerName = op.getStr("name");
+ String triggerName = op.getStr("trigger");
+ List<String> stageNames = op.getStrs("stage", Collections.emptyList());
+ String listenerClass = op.getStr("class");
+ List<String> beforeActions = op.getStrs("beforeAction", Collections.emptyList());
+ List<String> afterActions = op.getStrs("afterAction", Collections.emptyList());
+
+ if (listenerName == null || listenerName.trim().length() == 0) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The listener name cannot be null or empty");
+ }
+
+ Map<String, Object> autoScalingConf = zkReadAutoScalingConf(container.getZkController().getZkStateReader());
+ Map<String, Object> triggers = (Map<String, Object>) autoScalingConf.get("triggers");
+ if (triggers == null || !triggers.containsKey(triggerName)) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "A trigger with the name " + triggerName + " does not exist");
+ }
+ Map<String, Object> triggerProps = (Map<String, Object>) triggers.get(triggerName);
+
+ if (stageNames.isEmpty() && beforeActions.isEmpty() && afterActions.isEmpty()) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Either 'stage' or 'beforeAction' or 'afterAction' must be specified");
+ }
+
+ for (String stage : stageNames) {
+ try {
+ AutoScaling.TriggerStage.valueOf(stage);
+ } catch (IllegalArgumentException e) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid stage name: " + stage);
+ }
+ }
+
+ if (listenerClass == null || listenerClass.trim().length() == 0) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The 'class' of the listener cannot be null or empty");
+ }
+ // validate that we can load the listener class
+ // todo nocommit -- what about MemClassLoader?
+ try {
+ container.getResourceLoader().findClass(listenerClass, AutoScaling.TriggerListener.class);
+ } catch (Exception e) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Listener not found: " + listenerClass, e);
+ }
+
+ List<Map<String, String>> actions = (List<Map<String, String>>) triggerProps.get("actions");
+ Set<String> actionNames = new HashSet<>();
+ actionNames.addAll(beforeActions);
+ actionNames.addAll(afterActions);
+ for (Map<String, String> action : actions) {
+ actionNames.remove(action.get("name"));
+ }
+ if (!actionNames.isEmpty()) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The trigger '" + triggerName + "' does not have actions named: " + actionNames);
+ }
+
+ // todo - handle races between competing set-trigger and set-listener invocations
+ zkSetListener(container.getZkController().getZkStateReader(), listenerName, op.getValuesExcluding("name"));
+ rsp.getValues().add("result", "success");
+ }
+
+ private void zkSetListener(ZkStateReader reader, String listenerName, Map<String, Object> listenerProperties) throws KeeperException, InterruptedException {
+ while (true) {
+ Stat stat = new Stat();
+ ZkNodeProps loaded = null;
+ byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, stat, true);
+ loaded = ZkNodeProps.load(data);
+ Map<String, Object> listeners = (Map<String, Object>) loaded.get("listeners");
+ if (listeners == null) listeners = new HashMap<>(1);
+ if (listenerProperties != null) {
+ listeners.put(listenerName, listenerProperties);
+ } else {
+ listeners.remove(listenerName);
+ }
+ loaded = loaded.plus("listeners", listeners);
+ try {
+ reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(loaded), stat.getVersion(), true);
+ } catch (KeeperException.BadVersionException bve) {
+ // somebody else has changed the configuration so we must retry
+ continue;
}
+ break;
}
}
@@ -144,7 +251,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
try {
container.getResourceLoader().findClass(klass, TriggerAction.class);
} catch (Exception e) {
- throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error trying to find Action: " + klass, e);
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Action not found: " + klass, e);
}
}
@@ -154,6 +261,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
private void handleRemoveTrigger(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException {
String triggerName = op.getStr("name");
+ boolean removeListeners = op.getBoolean("removeListeners", false);
if (triggerName == null || triggerName.trim().length() == 0) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The trigger name cannot be null or empty");
@@ -163,6 +271,26 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
if (triggers == null || !triggers.containsKey(triggerName)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No trigger exists with name: " + triggerName);
}
+
+ Map<String, Map<String, Object>> listeners = (Map<String, Map<String, Object>>) autoScalingConf.get("listeners");
+ Set<String> activeListeners = new HashSet<>();
+ if (listeners != null) {
+ for (Map.Entry<String, Map<String, Object>> entry : listeners.entrySet()) {
+ Map<String, Object> listenerProps = entry.getValue();
+ if (triggerName.equals(listenerProps.get("trigger")) && !removeListeners) {
+ activeListeners.add(entry.getKey());
+ }
+ }
+ }
+ if (removeListeners) {
+ for (String activeListener : activeListeners) {
+ zkSetListener(container.getZkController().getZkStateReader(), activeListener, null);
+ }
+ } else if (!activeListeners.isEmpty()) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+ "No listeners should exist for trigger: " + triggerName + ". Found listeners: " + activeListeners);
+ }
+
zkSetTrigger(container.getZkController().getZkStateReader(), triggerName, null);
rsp.getValues().add("result", "success");
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb9d1bd/solr/core/src/resources/apispec/autoscaling.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/autoscaling.Commands.json b/solr/core/src/resources/apispec/autoscaling.Commands.json
index b374eac..e244c05 100644
--- a/solr/core/src/resources/apispec/autoscaling.Commands.json
+++ b/solr/core/src/resources/apispec/autoscaling.Commands.json
@@ -11,36 +11,36 @@
},
"commands": {
"set-trigger": {
- "type":"object",
+ "type": "object",
"description": "The set-trigger command allows you to add and update triggers on various system metrics",
"properties": {
"name": {
"type": "string",
"description": "The name of the trigger"
},
- "event" : {
- "type" : "string",
- "description" : "The event type on which to set a trigger"
+ "event": {
+ "type": "string",
+ "description": "The event type on which to set a trigger"
},
- "waitFor" : {
- "type" : "string",
- "description" : "The amount of time to wait after the trigger condition is satisfied before trigger is activated"
+ "waitFor": {
+ "type": "string",
+ "description": "The amount of time to wait after the trigger condition is satisfied before trigger is activated"
},
- "lowerBound" : {
- "type" : "number",
- "description" : "The lower bound of the condition below which the trigger is activated"
+ "lowerBound": {
+ "type": "number",
+ "description": "The lower bound of the condition below which the trigger is activated"
},
- "upperBound" : {
- "type" : "number",
- "description" : "The upper bound of the condition below which the trigger is activated"
+ "upperBound": {
+ "type": "number",
+ "description": "The upper bound of the condition below which the trigger is activated"
},
- "enabled" : {
- "type" : "boolean",
- "description" : "The state of the trigger"
+ "enabled": {
+ "type": "boolean",
+ "description": "The state of the trigger"
},
- "actions" : {
- "type" : "array",
- "description" : "The actions to be performed in sequence when the trigger is activated",
+ "actions": {
+ "type": "array",
+ "description": "The actions to be performed in sequence when the trigger is activated",
"items": {
"type": "object"
}
@@ -53,13 +53,78 @@
},
"remove-trigger": {
"description": "Remove a trigger",
- "type":"object",
- "properties" : {
- "name" : {
- "type" : "string",
- "description" : "The name of the trigger to be removed"
+ "type": "object",
+ "properties": {
+ "name": {
+ "type": "string",
+ "description": "The name of the trigger to be removed"
+ },
+ "removeListeners": {
+ "type": "boolean",
+ "description": "If true, all listeners of this triggers are deleted together with the trigger"
+ }
+ },
+ "required": [
+ "name"
+ ]
+ },
+ "set-listener": {
+ "description": "The set-listener command lets you add a listener to a trigger",
+ "type": "object",
+ "properties": {
+ "name": {
+ "type": "string",
+ "description": "The name of the listener"
+ },
+ "trigger": {
+ "type": "string",
+ "description": "The name of the trigger to listen to"
+ },
+ "stage": {
+ "type": "array",
+ "description": "The stage of the trigger for which to listen"
+ "items": {
+ "type" : "string"
+ }
+ },
+ "beforeAction": {
+ "type": "array",
+ "description": "The name of the action before which the listener should be notified"
+ "items": {
+ "type" : "string"
+ }
+ },
+ "afterAction": {
+ "type": "array",
+ "description": "The name of the action after which the listener should be notified"
+ "items": {
+ "type" : "string"
+ }
+ },
+ "class": {
+ "type": "string",
+ "description": "The listener class to be notified once the given stage of the given trigger is activated"
}
- }
+ },
+ "required": [
+ "name",
+ "trigger",
+ "class"
+ ],
+ "additionalProperties": true
+ },
+ "remove-listener": {
+ "description": "Remove a listener",
+ "type": "object",
+ "properties": {
+ "name": {
+ "type": "string",
+ "description": "The name of the listener to be removed"
+ }
+ },
+ "required": [
+ "name"
+ ]
}
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0fb9d1bd/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
index 9a1eb01..c08e6b4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
@@ -26,6 +26,7 @@ import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
@@ -53,7 +54,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
// todo nocommit -- add testing for the v2 path
// String path = random().nextBoolean() ? "/admin/autoscaling" : "/v2/cluster/autoscaling";
String path = "/admin/autoscaling";
- String addTriggerCommand = "{\n" +
+ String setTriggerCommand = "{\n" +
"\t\"set-trigger\" : {\n" +
"\t\t\"name\" : \"node_lost_trigger\",\n" +
"\t\t\"event\" : \"nodeLost\",\n" +
@@ -76,7 +77,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
"\t\t]\n" +
"\t}\n" +
"}";
- SolrRequest req = new AutoScalingRequest(SolrRequest.METHOD.POST, path, addTriggerCommand);
+ SolrRequest req = new AutoScalingRequest(SolrRequest.METHOD.POST, path, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
@@ -91,12 +92,100 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
assertEquals(4, nodeLostTrigger.size());
assertEquals("600", nodeLostTrigger.get("waitFor").toString());
+ setTriggerCommand = "{\n" +
+ "\t\"set-trigger\" : {\n" +
+ "\t\t\"name\" : \"node_lost_trigger\",\n" +
+ "\t\t\"event\" : \"nodeLost\",\n" +
+ "\t\t\"waitFor\" : \"20m\",\n" +
+ "\t\t\"enabled\" : \"false\",\n" +
+ "\t\t\"actions\" : [\n" +
+ "\t\t\t{\n" +
+ "\t\t\t\t\"name\" : \"compute_plan\",\n" +
+ "\t\t\t\t\"class\" : \"solr.ComputePlanAction\"\n" +
+ "\t\t\t},\n" +
+ "\t\t\t{\n" +
+ "\t\t\t\t\"name\" : \"execute_plan\",\n" +
+ "\t\t\t\t\"class\" : \"solr.ExecutePlanAction\"\n" +
+ "\t\t\t},\n" +
+ "\t\t\t{\n" +
+ "\t\t\t\t\"name\" : \"log_plan\",\n" +
+ "\t\t\t\t\"class\" : \"solr.LogPlanAction\",\n" +
+ "\t\t\t\t\"collection\" : \".system\"\n" +
+ "\t\t\t}\n" +
+ "\t\t]\n" +
+ "\t}\n" +
+ "}";
+ req = new AutoScalingRequest(SolrRequest.METHOD.POST, path, setTriggerCommand);
+ response = solrClient.request(req);
+ assertEquals(response.get("result").toString(), "success");
+ data = zkClient().getData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, null, null, true);
+ loaded = ZkNodeProps.load(data);
+ triggers = (Map<String, Object>) loaded.get("triggers");
+ assertNotNull(triggers);
+ assertEquals(1, triggers.size());
+ assertTrue(triggers.containsKey("node_lost_trigger"));
+ nodeLostTrigger = (Map<String, Object>) triggers.get("node_lost_trigger");
+ assertEquals(4, nodeLostTrigger.size());
+ assertEquals("1200", nodeLostTrigger.get("waitFor").toString());
+ assertEquals("false", nodeLostTrigger.get("enabled").toString());
+
+ String setListenerCommand = "{\n" +
+ "\t\"set-listener\" : \n" +
+ "\t\t{\n" +
+ "\t\t\t\"name\" : \"xyz\",\n" +
+ "\t\t\t\"trigger\" : \"node_lost_trigger\",\n" +
+ "\t\t\t\"stage\" : [\"STARTED\",\"ABORTED\",\"SUCCEEDED\"],\n" +
+ "\t\t\t\"beforeAction\" : \"execute_plan\",\n" +
+ "\t\t\t\"class\" : \"org.apache.solr.cloud.autoscaling.AutoScaling$HttpCallbackListener\",\n" +
+ "\t\t\t\"url\" : \"http://xyz.com/on_node_lost?node={$LOST_NODE_NAME}\"\n" +
+ "\t\t}\n" +
+ "}";
+ req = new AutoScalingRequest(SolrRequest.METHOD.POST, path, setListenerCommand);
+ response = solrClient.request(req);
+ assertEquals(response.get("result").toString(), "success");
+ data = zkClient().getData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, null, null, true);
+ loaded = ZkNodeProps.load(data);
+ Map<String, Object> listeners = (Map<String, Object>) loaded.get("listeners");
+ assertNotNull(listeners);
+ assertEquals(1, listeners.size());
+ assertTrue(listeners.containsKey("xyz"));
+ Map<String, Object> xyzListener = (Map<String, Object>) listeners.get("xyz");
+ assertEquals(5, xyzListener.size());
+ assertEquals("org.apache.solr.cloud.autoscaling.AutoScaling$HttpCallbackListener", xyzListener.get("class").toString());
+
String removeTriggerCommand = "{\n" +
"\t\"remove-trigger\" : {\n" +
"\t\t\"name\" : \"node_lost_trigger\"\n" +
"\t}\n" +
"}";
req = new AutoScalingRequest(SolrRequest.METHOD.POST, path, removeTriggerCommand);
+ try {
+ response = solrClient.request(req);
+ fail("Trying to remove trigger which has listeners registered should have failed");
+ } catch (HttpSolrClient.RemoteSolrException e) {
+ // expected
+ }
+
+ String removeListenerCommand = "{\n" +
+ "\t\"remove-listener\" : {\n" +
+ "\t\t\"name\" : \"xyz\"\n" +
+ "\t}\n" +
+ "}";
+ req = new AutoScalingRequest(SolrRequest.METHOD.POST, path, removeListenerCommand);
+ response = solrClient.request(req);
+ assertEquals(response.get("result").toString(), "success");
+ data = zkClient().getData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, null, null, true);
+ loaded = ZkNodeProps.load(data);
+ listeners = (Map<String, Object>) loaded.get("listeners");
+ assertNotNull(listeners);
+ assertEquals(0, listeners.size());
+
+ removeTriggerCommand = "{\n" +
+ "\t\"remove-trigger\" : {\n" +
+ "\t\t\"name\" : \"node_lost_trigger\"\n" +
+ "\t}\n" +
+ "}";
+ req = new AutoScalingRequest(SolrRequest.METHOD.POST, path, removeTriggerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
data = zkClient().getData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, null, null, true);
@@ -104,6 +193,25 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
triggers = (Map<String, Object>) loaded.get("triggers");
assertNotNull(triggers);
assertEquals(0, triggers.size());
+
+ setListenerCommand = "{\n" +
+ "\t\"set-listener\" : \n" +
+ "\t\t{\n" +
+ "\t\t\t\"name\" : \"xyz\",\n" +
+ "\t\t\t\"trigger\" : \"node_lost_trigger\",\n" +
+ "\t\t\t\"stage\" : [\"STARTED\",\"ABORTED\",\"SUCCEEDED\"],\n" +
+ "\t\t\t\"beforeAction\" : \"execute_plan\",\n" +
+ "\t\t\t\"class\" : \"org.apache.solr.cloud.autoscaling.AutoScaling$HttpCallbackListener\",\n" +
+ "\t\t\t\"url\" : \"http://xyz.com/on_node_lost?node={$LOST_NODE_NAME}\"\n" +
+ "\t\t}\n" +
+ "}";
+ req = new AutoScalingRequest(SolrRequest.METHOD.POST, path, removeListenerCommand);
+ try {
+ response = solrClient.request(req);
+ fail("Adding a listener on a non-existent trigger should have failed");
+ } catch (HttpSolrClient.RemoteSolrException e) {
+ // expected
+ }
}
static class AutoScalingRequest extends SolrRequest {