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 2018/04/03 14:00:02 UTC

[2/2] lucene-solr:master: SOLR-12095: AutoScalingHandler should validate triggers before updating zookeeper.

SOLR-12095: AutoScalingHandler should validate triggers before updating zookeeper.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/a14980c4
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/a14980c4
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/a14980c4

Branch: refs/heads/master
Commit: a14980c479608306aebb7255f7bb7eb64c476085
Parents: 1d6502c
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Apr 3 15:59:26 2018 +0200
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Apr 3 15:59:50 2018 +0200

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../solr/cloud/autoscaling/AutoScaling.java     |  40 +++++--
 .../cloud/autoscaling/AutoScalingHandler.java   |  32 ++++-
 .../cloud/autoscaling/ComputePlanAction.java    |  13 ++-
 .../cloud/autoscaling/HttpTriggerListener.java  |  12 +-
 .../autoscaling/InactiveShardPlanAction.java    |  18 +--
 .../solr/cloud/autoscaling/MetricTrigger.java   |  22 +++-
 .../cloud/autoscaling/NodeAddedTrigger.java     |  25 ++--
 .../solr/cloud/autoscaling/NodeLostTrigger.java |  20 ++--
 .../autoscaling/OverseerTriggerThread.java      |  12 +-
 .../cloud/autoscaling/ScheduledTrigger.java     |  37 +++---
 .../cloud/autoscaling/ScheduledTriggers.java    |   8 +-
 .../cloud/autoscaling/SearchRateTrigger.java    |  28 ++---
 .../cloud/autoscaling/SystemLogListener.java    |  10 +-
 .../solr/cloud/autoscaling/TriggerAction.java   |  24 +++-
 .../cloud/autoscaling/TriggerActionBase.java    |  50 +++++++-
 .../solr/cloud/autoscaling/TriggerBase.java     |  69 ++++++++---
 .../solr/cloud/autoscaling/TriggerListener.java |  20 +++-
 .../cloud/autoscaling/TriggerListenerBase.java  |  52 ++++++++-
 .../solr/cloud/autoscaling/TriggerUtils.java    |  88 ++++++++++++++
 .../autoscaling/TriggerValidationException.java |  74 ++++++++++++
 .../autoscaling/AutoScalingHandlerTest.java     | 117 +++++++++++++++++++
 .../autoscaling/ComputePlanActionTest.java      |  16 ++-
 .../autoscaling/ExecutePlanActionTest.java      |   9 +-
 .../autoscaling/HttpTriggerListenerTest.java    |  13 +--
 .../MetricTriggerIntegrationTest.java           |   5 +-
 .../cloud/autoscaling/MetricTriggerTest.java    |   6 +-
 .../NodeAddedTriggerIntegrationTest.java        |   5 +-
 .../cloud/autoscaling/NodeAddedTriggerTest.java |  52 +++++----
 .../NodeLostTriggerIntegrationTest.java         |   5 +-
 .../cloud/autoscaling/NodeLostTriggerTest.java  |  50 +++++---
 .../NodeMarkersRegistrationTest.java            |   5 +-
 .../autoscaling/RestoreTriggerStateTest.java    |   5 +-
 .../ScheduledMaintenanceTriggerTest.java        |   5 +-
 .../cloud/autoscaling/ScheduledTriggerTest.java |  12 +-
 .../SearchRateTriggerIntegrationTest.java       |   5 +-
 .../autoscaling/SearchRateTriggerTest.java      |   4 +-
 .../TriggerCooldownIntegrationTest.java         |   9 +-
 .../autoscaling/TriggerIntegrationTest.java     |  15 +--
 .../TriggerSetPropertiesIntegrationTest.java    |  19 +--
 .../autoscaling/sim/TestComputePlanAction.java  |  18 ++-
 .../autoscaling/sim/TestExecutePlanAction.java  |   2 +-
 .../cloud/autoscaling/sim/TestLargeCluster.java |   6 +-
 .../autoscaling/sim/TestNodeAddedTrigger.java   |  43 ++++---
 .../autoscaling/sim/TestNodeLostTrigger.java    |  39 +++++--
 .../autoscaling/sim/TestTriggerIntegration.java |  19 +--
 46 files changed, 883 insertions(+), 257 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index d7a01f6..2b40c3b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -157,6 +157,8 @@ Other Changes
 
 * SOLR-12144: SOLR_LOG_PRESTART_ROTATION now defaults to false, we leverage log4j2 for log rotation on startup (janhoy)
 
+* SOLR-12095: AutoScalingHandler validates trigger configurations before updating Zookeeper. (ab)
+
 ==================  7.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/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 a3a9aa8..68282a7 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
@@ -106,10 +106,19 @@ public class AutoScaling {
     void restoreState();
 
     /**
+     * Called when trigger is created but before it's initialized or scheduled for use.
+     * This method should also verify that the trigger configuration parameters are correct. It may
+     * be called multiple times.
+     * @param properties configuration properties
+     * @throws TriggerValidationException contains details of invalid configuration parameters.
+     */
+    void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException;
+
+    /**
      * Called before a trigger is scheduled. Any heavy object creation or initialisation should
      * be done in this method instead of the Trigger's constructor.
      */
-    void init();
+    void init() throws Exception;
   }
 
   /**
@@ -118,7 +127,7 @@ public class AutoScaling {
   public static abstract class TriggerFactory implements Closeable {
     protected boolean isClosed = false;
 
-    public abstract Trigger create(TriggerEventType type, String name, Map<String, Object> props);
+    public abstract Trigger create(TriggerEventType type, String name, Map<String, Object> props) throws TriggerValidationException;
 
     @Override
     public void close() throws IOException {
@@ -144,24 +153,38 @@ public class AutoScaling {
     }
 
     @Override
-    public synchronized Trigger create(TriggerEventType type, String name, Map<String, Object> props) {
+    public synchronized Trigger create(TriggerEventType type, String name, Map<String, Object> props) throws TriggerValidationException {
       if (isClosed) {
         throw new AlreadyClosedException("TriggerFactory has already been closed, cannot create new triggers");
       }
+      if (type == null) {
+        throw new IllegalArgumentException("Trigger type must not be null");
+      }
+      if (name == null || name.isEmpty()) {
+        throw new IllegalArgumentException("Trigger name must not be empty");
+      }
+      Trigger t;
       switch (type) {
         case NODEADDED:
-          return new NodeAddedTrigger(name, props, loader, cloudManager);
+          t = new NodeAddedTrigger(name);
+          break;
         case NODELOST:
-          return new NodeLostTrigger(name, props, loader, cloudManager);
+          t = new NodeLostTrigger(name);
+        break;
         case SEARCHRATE:
-          return new SearchRateTrigger(name, props, loader, cloudManager);
+          t = new SearchRateTrigger(name);
+        break;
         case METRIC:
-          return new MetricTrigger(name, props, loader, cloudManager);
+          t = new MetricTrigger(name);
+        break;
         case SCHEDULED:
-          return new ScheduledTrigger(name, props, loader, cloudManager);
+          t = new ScheduledTrigger(name);
+        break;
         default:
           throw new IllegalArgumentException("Unknown event type: " + type + " in trigger: " + name);
       }
+      t.configure(loader, cloudManager, props);
+      return t;
     }
 
   }
@@ -210,4 +233,5 @@ public class AutoScaling {
           "    }";
 
   public static final Map<String, Object> SCHEDULED_MAINTENANCE_TRIGGER_PROPS = (Map) Utils.fromJSONString(SCHEDULED_MAINTENANCE_TRIGGER_DSL);
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/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 5126e2e..a69451e 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
@@ -50,6 +50,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.SolrResourceLoader;
@@ -79,6 +80,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final SolrCloudManager cloudManager;
   protected final SolrResourceLoader loader;
+  protected final AutoScaling.TriggerFactory triggerFactory;
   private final List<Map<String, String>> DEFAULT_ACTIONS = new ArrayList<>(3);
   private static Set<String> singletonCommands = Stream.of("set-cluster-preferences", "set-cluster-policy")
       .collect(collectingAndThen(toSet(), Collections::unmodifiableSet));
@@ -88,6 +90,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
   public AutoScalingHandler(SolrCloudManager cloudManager, SolrResourceLoader loader) {
     this.cloudManager = cloudManager;
     this.loader = loader;
+    this.triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, cloudManager);
     this.timeSource = cloudManager.getTimeSource();
     Map<String, String> map = new HashMap<>(2);
     map.put(NAME, "compute_plan");
@@ -455,14 +458,26 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
     }
     if (op.hasError()) return currentConfig;
 
+    AutoScalingConfig.TriggerListenerConfig listenerConfig = new AutoScalingConfig.TriggerListenerConfig(listenerName, op.getValuesExcluding("name"));
+
     // validate that we can load the listener class
     // todo allow creation from blobstore
+    TriggerListener listener = null;
     try {
-      loader.findClass(listenerClass, TriggerListener.class);
+      listener = loader.newInstance(listenerClass, TriggerListener.class);
+      listener.configure(loader, cloudManager, listenerConfig);
+    } catch (TriggerValidationException e) {
+      log.warn("invalid listener configuration", e);
+      op.addError("invalid listener configuration: " + e.toString());
+      return currentConfig;
     } catch (Exception e) {
       log.warn("error loading listener class ", e);
       op.addError("Listener not found: " + listenerClass + ". error message:" + e.getMessage());
       return currentConfig;
+    } finally {
+      if (listener != null) {
+        IOUtils.closeQuietly(listener);
+      }
     }
 
     Set<String> actionNames = new HashSet<>();
@@ -475,9 +490,8 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
       op.addError("The trigger '" + triggerName + "' does not have actions named: " + actionNames);
       return currentConfig;
     }
-    AutoScalingConfig.TriggerListenerConfig listener = new AutoScalingConfig.TriggerListenerConfig(listenerName, op.getValuesExcluding("name"));
     // todo - handle races between competing set-trigger and set-listener invocations
-    currentConfig = currentConfig.withTriggerListenerConfig(listener);
+    currentConfig = currentConfig.withTriggerListenerConfig(listenerConfig);
     return currentConfig;
   }
 
@@ -531,6 +545,18 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
       }
     }
     AutoScalingConfig.TriggerConfig trigger = new AutoScalingConfig.TriggerConfig(triggerName, opCopy.getValuesExcluding("name"));
+    // validate trigger config
+    AutoScaling.Trigger t = null;
+    try {
+      t = triggerFactory.create(trigger.event, trigger.name, trigger.properties);
+    } catch (Exception e) {
+      op.addError("Error validating trigger config " + trigger.name + ": " + e.toString());
+      return currentConfig;
+    } finally {
+      if (t != null) {
+        IOUtils.closeQuietly(t);
+      }
+    }
     currentConfig = currentConfig.withTriggerConfig(trigger);
     // check that there's a default SystemLogListener, unless user specified another one
     return withSystemLogListener(currentConfig, triggerName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
index dc3cfd5..8f3175c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
@@ -40,6 +40,7 @@ import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.SolrResourceLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -55,10 +56,16 @@ public class ComputePlanAction extends TriggerActionBase {
 
   Set<String> collections = new HashSet<>();
 
+  public ComputePlanAction() {
+    super();
+    TriggerUtils.validProperties(validProperties, "collections");
+  }
+
+
   @Override
-  public void init(Map<String, String> args) {
-    super.init(args);
-    String colString = args.get("collections");
+  public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
+    super.configure(loader, cloudManager, properties);
+    String colString = (String) properties.get("collections");
     if (colString != null && !colString.isEmpty()) {
       collections.addAll(StrUtils.splitSmart(colString, ','));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
index b35d0be..e620966 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
@@ -29,6 +29,7 @@ import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.PropertiesUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -65,9 +66,16 @@ public class HttpTriggerListener extends TriggerListenerBase {
   private int timeout = HttpClientUtil.DEFAULT_CONNECT_TIMEOUT;
   private boolean followRedirects;
 
+  public HttpTriggerListener() {
+    super();
+    TriggerUtils.requiredProperties(requiredProperties, validProperties, "url");
+    TriggerUtils.validProperties(validProperties, "payload", "contentType", "timeout", "followRedirects");
+    validPropertyPrefixes.add("header.");
+  }
+
   @Override
-  public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
-    super.init(cloudManager, config);
+  public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
+    super.configure(loader, cloudManager, config);
     urlTemplate = (String)config.properties.get("url");
     payloadTemplate = (String)config.properties.get("payload");
     contentType = (String)config.properties.get("contentType");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveShardPlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveShardPlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveShardPlanAction.java
index 5e962a8..cb561f5 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveShardPlanAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/InactiveShardPlanAction.java
@@ -29,6 +29,7 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.core.SolrResourceLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,19 +49,22 @@ public class InactiveShardPlanAction extends TriggerActionBase {
 
   private int cleanupTTL;
 
+  public InactiveShardPlanAction() {
+    super();
+    TriggerUtils.validProperties(validProperties, TTL_PROP);
+  }
+
   @Override
-  public void init(Map<String, String> args) {
-    super.init(args);
-    String cleanupStr = args.getOrDefault(TTL_PROP, String.valueOf(DEFAULT_TTL_SECONDS));
+  public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
+    super.configure(loader, cloudManager, properties);
+    String cleanupStr = String.valueOf(properties.getOrDefault(TTL_PROP, String.valueOf(DEFAULT_TTL_SECONDS)));
     try {
       cleanupTTL = Integer.parseInt(cleanupStr);
     } catch (Exception e) {
-      log.warn("Invalid " + TTL_PROP + " value: '" + cleanupStr + "', using default " + DEFAULT_TTL_SECONDS);
-      cleanupTTL = DEFAULT_TTL_SECONDS;
+      throw new TriggerValidationException(getName(), TTL_PROP, "invalid value '" + cleanupStr + "': " + e.toString());
     }
     if (cleanupTTL < 0) {
-      log.warn("Invalid " + TTL_PROP + " value: '" + cleanupStr + "', using default " + DEFAULT_TTL_SECONDS);
-      cleanupTTL = DEFAULT_TTL_SECONDS;
+      throw new TriggerValidationException(getName(), TTL_PROP, "invalid value '" + cleanupStr + "', should be > 0. ");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
index e0dd253..9fdf8dc 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
@@ -53,21 +53,31 @@ import static org.apache.solr.common.params.AutoScalingParams.PREFERRED_OP;
 public class MetricTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final String metric;
-  private final Number above, below;
-  private final String collection, shard, node, preferredOp;
+  private String metric;
+  private Number above, below;
+  private String collection, shard, node, preferredOp;
 
   private final Map<String, Long> lastNodeEvent = new ConcurrentHashMap<>();
 
-  public MetricTrigger(String name, Map<String, Object> properties, SolrResourceLoader loader, SolrCloudManager cloudManager) {
-    super(TriggerEventType.METRIC, name, properties, loader, cloudManager);
+  public MetricTrigger(String name) {
+    super(TriggerEventType.METRIC, name);
+    TriggerUtils.requiredProperties(requiredProperties, validProperties, METRIC);
+    TriggerUtils.validProperties(validProperties, ABOVE, BELOW, PREFERRED_OP,
+        AutoScalingParams.COLLECTION,
+        AutoScalingParams.SHARD,
+        AutoScalingParams.NODE);
+  }
+
+  @Override
+  public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
+    super.configure(loader, cloudManager, properties);
     this.metric = (String) properties.get(METRIC);
     this.above = (Number) properties.get(ABOVE);
     this.below = (Number) properties.get(BELOW);
     this.collection = (String) properties.getOrDefault(AutoScalingParams.COLLECTION, Policy.ANY);
     shard = (String) properties.getOrDefault(AutoScalingParams.SHARD, Policy.ANY);
     if (collection.equals(Policy.ANY) && !shard.equals(Policy.ANY)) {
-      throw new IllegalArgumentException("When 'shard' is other than #ANY then collection name must be also other than #ANY");
+      throw new TriggerValidationException("shard", "When 'shard' is other than #ANY then collection name must be also other than #ANY");
     }
     node = (String) properties.getOrDefault(AutoScalingParams.NODE, Policy.ANY);
     preferredOp = (String) properties.getOrDefault(PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
index 7f389e1..ca28dc4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeAddedTrigger.java
@@ -29,12 +29,9 @@ import java.util.NoSuchElementException;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.core.SolrResourceLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,22 +41,20 @@ import org.slf4j.LoggerFactory;
 public class NodeAddedTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private Set<String> lastLiveNodes;
+  private Set<String> lastLiveNodes = new HashSet<>();
 
   private Map<String, Long> nodeNameVsTimeAdded = new HashMap<>();
 
-  public NodeAddedTrigger(String name, Map<String, Object> properties,
-                          SolrResourceLoader loader,
-                          SolrCloudManager cloudManager) {
-    super(TriggerEventType.NODEADDED, name, properties, loader, cloudManager);
-    lastLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
-    log.debug("NodeAddedTrigger {} - Initial livenodes: {}", name, lastLiveNodes);
-    log.debug("NodeAddedTrigger {} instantiated with properties: {}", name, properties);
+  public NodeAddedTrigger(String name) {
+    super(TriggerEventType.NODEADDED, name);
   }
 
   @Override
-  public void init() {
+  public void init() throws Exception {
     super.init();
+    lastLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
+    log.debug("NodeAddedTrigger {} - Initial livenodes: {}", name, lastLiveNodes);
+    log.debug("NodeAddedTrigger {} instantiated with properties: {}", name, properties);
     // pick up added nodes for which marker paths were created
     try {
       List<String> added = stateManager.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
@@ -85,8 +80,10 @@ public class NodeAddedTrigger extends TriggerBase {
     if (old instanceof NodeAddedTrigger) {
       NodeAddedTrigger that = (NodeAddedTrigger) old;
       assert this.name.equals(that.name);
-      this.lastLiveNodes = new HashSet<>(that.lastLiveNodes);
-      this.nodeNameVsTimeAdded = new HashMap<>(that.nodeNameVsTimeAdded);
+      this.lastLiveNodes.clear();
+      this.lastLiveNodes.addAll(that.lastLiveNodes);
+      this.nodeNameVsTimeAdded.clear();
+      this.nodeNameVsTimeAdded.putAll(that.nodeNameVsTimeAdded);
     } else  {
       throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
           "Unable to restore state from an unknown type of trigger");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
index 2981a48..6dfccb2 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/NodeLostTrigger.java
@@ -43,21 +43,19 @@ import org.slf4j.LoggerFactory;
 public class NodeLostTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private Set<String> lastLiveNodes;
+  private Set<String> lastLiveNodes = new HashSet<>();
 
   private Map<String, Long> nodeNameVsTimeRemoved = new HashMap<>();
 
-  public NodeLostTrigger(String name, Map<String, Object> properties,
-                         SolrResourceLoader loader,
-                         SolrCloudManager dataProvider) {
-    super(TriggerEventType.NODELOST, name, properties, loader, dataProvider);
-    lastLiveNodes = new HashSet<>(dataProvider.getClusterStateProvider().getLiveNodes());
-    log.debug("NodeLostTrigger {} - Initial livenodes: {}", name, lastLiveNodes);
+  public NodeLostTrigger(String name) {
+    super(TriggerEventType.NODELOST, name);
   }
 
   @Override
-  public void init() {
+  public void init() throws Exception {
     super.init();
+    lastLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
+    log.debug("NodeLostTrigger {} - Initial livenodes: {}", name, lastLiveNodes);
     // pick up lost nodes for which marker paths were created
     try {
       List<String> lost = stateManager.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
@@ -82,8 +80,10 @@ public class NodeLostTrigger extends TriggerBase {
     if (old instanceof NodeLostTrigger) {
       NodeLostTrigger that = (NodeLostTrigger) old;
       assert this.name.equals(that.name);
-      this.lastLiveNodes = new HashSet<>(that.lastLiveNodes);
-      this.nodeNameVsTimeRemoved = new HashMap<>(that.nodeNameVsTimeRemoved);
+      this.lastLiveNodes.clear();
+      this.lastLiveNodes.addAll(that.lastLiveNodes);
+      this.nodeNameVsTimeRemoved.clear();
+      this.nodeNameVsTimeRemoved.putAll(that.nodeNameVsTimeRemoved);
     } else  {
       throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
           "Unable to restore state from an unknown type of trigger");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/OverseerTriggerThread.java
----------------------------------------------------------------------
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 874122d..ece4c4c 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
@@ -230,7 +230,11 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
           if (entry.getValue().getEventType().equals(TriggerEventType.NODEADDED)) {
             cleanOldNodeAddedMarkers = false;
           }
-          scheduledTriggers.add(entry.getValue());
+          try {
+            scheduledTriggers.add(entry.getValue());
+          } catch (Exception e) {
+            log.warn("Exception initializing trigger " + entry.getKey() + ", configuration ignored", e);
+          }
         }
       } catch (AlreadyClosedException e) {
         // this _should_ mean that we're closing, complain loudly if that's not the case
@@ -382,7 +386,11 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
       AutoScalingConfig.TriggerConfig cfg = entry.getValue();
       TriggerEventType eventType = cfg.event;
       String triggerName = entry.getKey();
-      triggerMap.put(triggerName, triggerFactory.create(eventType, triggerName, cfg.properties));
+      try {
+        triggerMap.put(triggerName, triggerFactory.create(eventType, triggerName, cfg.properties));
+      } catch (TriggerValidationException e) {
+        log.warn("Error in trigger '" + triggerName + "' configuration, trigger config ignored: " + cfg, e);
+      }
     }
     return triggerMap;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
index bbc9c30..5e25542 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
@@ -33,6 +33,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.DateMathParser;
@@ -52,20 +53,25 @@ public class ScheduledTrigger extends TriggerBase {
   private static final String LAST_RUN_AT = "lastRunAt";
   static final String ACTUAL_EVENT_TIME = "actualEventTime";
 
-  private final String everyStr;
+  private String everyStr;
 
-  private final String graceDurationStr;
+  private String graceDurationStr;
 
-  private final String preferredOp;
+  private String preferredOp;
 
-  private final TimeZone timeZone;
+  private TimeZone timeZone;
 
   private Instant lastRunAt;
 
-  public ScheduledTrigger(String name, Map<String, Object> properties,
-                          SolrResourceLoader loader, SolrCloudManager cloudManager) {
-    super(TriggerEventType.SCHEDULED, name, properties, loader, cloudManager);
+  public ScheduledTrigger(String name) {
+    super(TriggerEventType.SCHEDULED, name);
+    TriggerUtils.requiredProperties(requiredProperties, validProperties, "startTime");
+    TriggerUtils.validProperties(validProperties, "timeZone", "every", "graceDuration", AutoScalingParams.PREFERRED_OP);
+  }
 
+  @Override
+  public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
+    super.configure(loader, cloudManager, properties);
     String timeZoneStr = (String) properties.get("timeZone");
     this.timeZone = TimeZoneUtils.parseTimezone(timeZoneStr); // defaults to UTC
 
@@ -90,18 +96,17 @@ public class ScheduledTrigger extends TriggerBase {
     this.lastRunAt = startTime;
   }
 
-  private Instant parseStartTime(Date now, String startTimeStr, String timeZoneStr) {
-    if (startTimeStr == null) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Parameter 'startTime' cannot be null");
-    }
+  private Instant parseStartTime(Date now, String startTimeStr, String timeZoneStr) throws TriggerValidationException {
     try {
       // try parsing startTime as an ISO-8601 date time string
       return DateMathParser.parseMath(now, startTimeStr).toInstant();
     } catch (SolrException e) {
-      if (e.code() != SolrException.ErrorCode.BAD_REQUEST.code)  throw e;
+      if (e.code() != SolrException.ErrorCode.BAD_REQUEST.code) {
+        throw new TriggerValidationException("startTime", "error parsing value '" + startTimeStr + "': " + e.toString());
+      }
     }
     if (timeZoneStr == null)  {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+      throw new TriggerValidationException("timeZone",
           "Either 'startTime' should be an ISO-8601 date time string or 'timeZone' must be not be null");
     }
     TimeZone timeZone = TimeZone.getTimeZone(timeZoneStr);
@@ -111,7 +116,11 @@ public class ScheduledTrigger extends TriggerBase {
         .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0)
         .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0)
         .toFormatter(Locale.ROOT).withZone(timeZone.toZoneId());
-    return Instant.from(dateTimeFormatter.parse(startTimeStr));
+    try {
+      return Instant.from(dateTimeFormatter.parse(startTimeStr));
+    } catch (Exception e) {
+      throw new TriggerValidationException("startTime", "error parsing startTime '" + startTimeStr + "': " + e.toString());
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/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 28efe92..c125209 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
@@ -197,7 +197,7 @@ public class ScheduledTriggers implements Closeable {
    * @param newTrigger the trigger to be managed
    * @throws AlreadyClosedException if this class has already been closed
    */
-  public synchronized void add(AutoScaling.Trigger newTrigger) {
+  public synchronized void add(AutoScaling.Trigger newTrigger) throws Exception {
     if (isClosed) {
       throw new AlreadyClosedException("ScheduledTriggers has been closed and cannot be used anymore");
     }
@@ -685,7 +685,8 @@ public class ScheduledTriggers implements Closeable {
             }
             if (listener != null) {
               try {
-                listener.init(cloudManager, config);
+                listener.configure(loader, cloudManager, config);
+                listener.init();
                 listenersPerName.put(config.name, listener);
               } catch (Exception e) {
                 log.warn("Error initializing TriggerListener " + config, e);
@@ -769,6 +770,9 @@ public class ScheduledTriggers implements Closeable {
       updateLock.lock();
       try {
         for (TriggerListener listener : getTriggerListeners(trigger, stage)) {
+          if (!listener.isEnabled()) {
+            continue;
+          }
           if (actionName != null) {
             AutoScalingConfig.TriggerListenerConfig config = listener.getConfig();
             if (stage == TriggerEventProcessorStage.BEFORE_ACTION) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
index 814c676..00bc6d8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
@@ -49,43 +49,43 @@ import org.slf4j.LoggerFactory;
 public class SearchRateTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private final String handler;
-  private final String collection;
-  private final String shard;
-  private final String node;
-  private final double rate;
+  private String handler;
+  private String collection;
+  private String shard;
+  private String node;
+  private double rate;
   private final Map<String, Long> lastCollectionEvent = new ConcurrentHashMap<>();
   private final Map<String, Long> lastNodeEvent = new ConcurrentHashMap<>();
   private final Map<String, Long> lastShardEvent = new ConcurrentHashMap<>();
   private final Map<String, Long> lastReplicaEvent = new ConcurrentHashMap<>();
   private final Map<String, Object> state = new HashMap<>();
 
-  public SearchRateTrigger(String name, Map<String, Object> properties,
-                           SolrResourceLoader loader,
-                           SolrCloudManager cloudManager) {
-    super(TriggerEventType.SEARCHRATE, name, properties, loader, cloudManager);
+  public SearchRateTrigger(String name) {
+    super(TriggerEventType.SEARCHRATE, name);
     this.state.put("lastCollectionEvent", lastCollectionEvent);
     this.state.put("lastNodeEvent", lastNodeEvent);
     this.state.put("lastShardEvent", lastShardEvent);
     this.state.put("lastReplicaEvent", lastReplicaEvent);
+    TriggerUtils.requiredProperties(requiredProperties, validProperties, "rate");
+  }
 
+  @Override
+  public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
+    super.configure(loader, cloudManager, properties);
     // parse config options
     collection = (String)properties.getOrDefault(AutoScalingParams.COLLECTION, Policy.ANY);
     shard = (String)properties.getOrDefault(AutoScalingParams.SHARD, Policy.ANY);
     if (collection.equals(Policy.ANY) && !shard.equals(Policy.ANY)) {
-      throw new IllegalArgumentException("When 'shard' is other than #ANY then collection name must be also other than #ANY");
+      throw new TriggerValidationException("shard", "When 'shard' is other than #ANY then collection name must be also other than #ANY");
     }
     node = (String)properties.getOrDefault(AutoScalingParams.NODE, Policy.ANY);
     handler = (String)properties.getOrDefault(AutoScalingParams.HANDLER, "/select");
 
-    if (properties.get("rate") == null) {
-      throw new IllegalArgumentException("No 'rate' specified in configuration");
-    }
     String rateString = String.valueOf(properties.get("rate"));
     try {
       rate = Double.parseDouble(rateString);
     } catch (Exception e) {
-      throw new IllegalArgumentException("Invalid 'rate' configuration value: '" + rateString + "'", e);
+      throw new TriggerValidationException(name, "rate", "Invalid 'rate' configuration value: '" + rateString + "': " + e.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
index ea34112..c6f0e68 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SystemLogListener.java
@@ -41,6 +41,7 @@ import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.IdUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -69,21 +70,16 @@ public class SystemLogListener extends TriggerListenerBase {
   public static final String DOC_TYPE = "autoscaling_event";
 
   private String collection = CollectionAdminParams.SYSTEM_COLL;
-  private boolean enabled = true;
 
   @Override
-  public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
-    super.init(cloudManager, config);
+  public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
+    super.configure(loader, cloudManager, config);
     collection = (String)config.properties.getOrDefault(CollectionAdminParams.COLLECTION, CollectionAdminParams.SYSTEM_COLL);
-    enabled = Boolean.parseBoolean(String.valueOf(config.properties.getOrDefault("enabled", true)));
   }
 
   @Override
   public void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context,
                Throwable error, String message) throws Exception {
-    if (!enabled) {
-      return;
-    }
     try {
       SolrInputDocument doc = new SolrInputDocument();
       doc.addField(CommonParams.TYPE, DOC_TYPE);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerAction.java
index df2022e..b873ee6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerAction.java
@@ -18,13 +18,33 @@
 package org.apache.solr.cloud.autoscaling;
 
 import java.io.Closeable;
+import java.util.Map;
 
-import org.apache.solr.util.plugin.MapInitializedPlugin;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.core.SolrResourceLoader;
 
 /**
  * Interface for actions performed in response to a trigger being activated
  */
-public interface TriggerAction extends MapInitializedPlugin, Closeable {
+public interface TriggerAction extends Closeable {
+
+  /**
+   * Called when action is created but before it's initialized and used.
+   * This method should also verify that the configuration parameters are correct.
+   * It may be called multiple times.
+   * @param loader loader to use for instantiating sub-components
+   * @param cloudManager current instance of SolrCloudManager
+   * @param properties configuration properties
+   * @throws TriggerValidationException contains details of invalid configuration parameters.
+   */
+  void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException;
+
+  /**
+   * Called before an action is first used. Any heavy object creation or initialization should
+   * be done in this method instead of the constructor or {@link #configure(SolrResourceLoader, SolrCloudManager, Map)} method.
+   */
+  void init() throws Exception;
+
   String getName();
 
   void process(TriggerEvent event, ActionContext context) throws Exception;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionBase.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionBase.java
index 75c4a87..7a9f34b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionBase.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerActionBase.java
@@ -17,19 +17,44 @@
 package org.apache.solr.cloud.autoscaling;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.core.SolrResourceLoader;
 
 /**
  * Base class for {@link TriggerAction} implementations.
  */
 public abstract class TriggerActionBase implements TriggerAction {
 
-  protected Map<String, String> initArgs;
+  protected Map<String, Object> properties = new HashMap<>();
+  protected SolrResourceLoader loader;
+  protected SolrCloudManager cloudManager;
+  /**
+   * Set of valid property names. Subclasses may add to this set
+   * using {@link TriggerUtils#validProperties(Set, String...)}
+   */
+  protected final Set<String> validProperties = new HashSet<>();
+  /**
+   * Set of required property names. Subclasses may add to this set
+   * using {@link TriggerUtils#requiredProperties(Set, Set, String...)}
+   * (required properties are also valid properties).
+   */
+  protected final Set<String> requiredProperties = new HashSet<>();
+
+  protected TriggerActionBase() {
+    // not strictly needed here because they are already checked during instantiation
+    TriggerUtils.validProperties(validProperties, "name", "class");
+  }
 
   @Override
   public String getName() {
-    if (initArgs != null) {
-      return initArgs.get("name");
+    String name = (String) properties.get("name");
+    if (name != null) {
+      return name;
     } else {
       return getClass().getSimpleName();
     }
@@ -41,7 +66,22 @@ public abstract class TriggerActionBase implements TriggerAction {
   }
 
   @Override
-  public void init(Map<String, String> args) {
-    this.initArgs = args;
+  public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
+    this.loader = loader;
+    this.cloudManager = cloudManager;
+    if (properties != null) {
+      this.properties.putAll(properties);
+    }
+    // validate the config
+    Map<String, String> results = new HashMap<>();
+    TriggerUtils.checkProperties(this.properties, results, requiredProperties, validProperties);
+    if (!results.isEmpty()) {
+      throw new TriggerValidationException(getName(), results);
+    }
+  }
+
+  @Override
+  public void init() throws Exception {
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java
index 04fa37e..12d95bc 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerBase.java
@@ -21,9 +21,11 @@ import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.lucene.util.IOUtils;
@@ -50,39 +52,75 @@ public abstract class TriggerBase implements AutoScaling.Trigger {
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   protected final String name;
-  protected final SolrCloudManager cloudManager;
-  protected final DistribStateManager stateManager;
+  protected SolrCloudManager cloudManager;
+  protected SolrResourceLoader loader;
+  protected DistribStateManager stateManager;
   protected final Map<String, Object> properties = new HashMap<>();
+  /**
+   * Set of valid property names. Subclasses may add to this set
+   * using {@link TriggerUtils#validProperties(Set, String...)}
+   */
+  protected final Set<String> validProperties = new HashSet<>();
+  /**
+   * Set of required property names. Subclasses may add to this set
+   * using {@link TriggerUtils#requiredProperties(Set, Set, String...)}
+   * (required properties are also valid properties).
+   */
+  protected final Set<String> requiredProperties = new HashSet<>();
   protected final TriggerEventType eventType;
-  protected final int waitForSecond;
+  protected int waitForSecond;
   protected Map<String,Object> lastState;
   protected final AtomicReference<AutoScaling.TriggerEventProcessor> processorRef = new AtomicReference<>();
-  protected final List<TriggerAction> actions;
-  protected final boolean enabled;
+  protected List<TriggerAction> actions;
+  protected boolean enabled;
   protected boolean isClosed;
 
 
-  protected TriggerBase(TriggerEventType eventType, String name, Map<String, Object> properties, SolrResourceLoader loader, SolrCloudManager cloudManager) {
+  protected TriggerBase(TriggerEventType eventType, String name) {
     this.eventType = eventType;
     this.name = name;
+
+    // subclasses may modify this set to include other supported properties
+    TriggerUtils.validProperties(validProperties, "name", "class", "event", "enabled", "waitFor", "actions");
+  }
+
+  @Override
+  public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
     this.cloudManager = cloudManager;
+    this.loader = loader;
     this.stateManager = cloudManager.getDistribStateManager();
     if (properties != null) {
       this.properties.putAll(properties);
     }
     this.enabled = Boolean.parseBoolean(String.valueOf(this.properties.getOrDefault("enabled", "true")));
     this.waitForSecond = ((Number) this.properties.getOrDefault("waitFor", -1L)).intValue();
-    List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
+    List<Map<String, Object>> o = (List<Map<String, Object>>) properties.get("actions");
     if (o != null && !o.isEmpty()) {
       actions = new ArrayList<>(3);
-      for (Map<String, String> map : o) {
-        TriggerAction action = loader.newInstance(map.get("class"), TriggerAction.class);
+      for (Map<String, Object> map : o) {
+        TriggerAction action = null;
+        try {
+          action = loader.newInstance((String)map.get("class"), TriggerAction.class);
+        } catch (Exception e) {
+          throw new TriggerValidationException("action", "exception creating action " + map + ": " + e.toString());
+        }
+        action.configure(loader, cloudManager, map);
         actions.add(action);
       }
     } else {
       actions = Collections.emptyList();
     }
 
+
+    Map<String, String> results = new HashMap<>();
+    TriggerUtils.checkProperties(this.properties, results, requiredProperties, validProperties);
+    if (!results.isEmpty()) {
+      throw new TriggerValidationException(name, results);
+    }
+  }
+
+  @Override
+  public void init() throws Exception {
     try {
       if (!stateManager.hasData(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH)) {
         stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
@@ -91,17 +129,10 @@ public abstract class TriggerBase implements AutoScaling.Trigger {
       // ignore
     } catch (InterruptedException | KeeperException | IOException e) {
       LOG.warn("Exception checking ZK path " + ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH, e);
+      throw e;
     }
-  }
-
-  @Override
-  public void init() {
-    List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
-    if (o != null && !o.isEmpty()) {
-      for (int i = 0; i < o.size(); i++) {
-        Map<String, String> map = o.get(i);
-        actions.get(i).init(map);
-      }
+    for (TriggerAction action : actions) {
+      action.init();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java
index 663bf5a..234387f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java
@@ -21,6 +21,7 @@ import java.io.Closeable;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
+import org.apache.solr.core.SolrResourceLoader;
 
 /**
  * Implementations of this interface are notified of stages in event processing that they were
@@ -28,7 +29,24 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage
  */
 public interface TriggerListener extends Closeable {
 
-  void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) throws Exception;
+  /**
+   * Called when listener is created but before it's initialized and used.
+   * This method should also verify that the configuration parameters are correct.
+   * It may be called multiple times.
+   * @param loader loader to use for instantiating sub-components
+   * @param cloudManager current instance of SolrCloudManager
+   * @param config coniguration
+   * @throws TriggerValidationException contains details of invalid configuration parameters.
+   */
+  void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException;
+
+  /**
+   * If this method returns false then the listener's {@link #onEvent(TriggerEvent, TriggerEventProcessorStage, String, ActionContext, Throwable, String)}
+   * method should not be called.
+   */
+  boolean isEnabled();
+
+  void init() throws Exception;
 
   AutoScalingConfig.TriggerListenerConfig getConfig();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
index 884ad97..7a323c7 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
@@ -17,9 +17,14 @@
 package org.apache.solr.cloud.autoscaling;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.core.SolrResourceLoader;
 
 /**
  * Base class for implementations of {@link TriggerListener}.
@@ -28,11 +33,46 @@ public abstract class TriggerListenerBase implements TriggerListener {
 
   protected AutoScalingConfig.TriggerListenerConfig config;
   protected SolrCloudManager cloudManager;
+  protected SolrResourceLoader loader;
+  protected boolean enabled;
+  /**
+   * Set of valid property names. Subclasses may add to this set
+   * using {@link TriggerUtils#validProperties(Set, String...)}
+   */
+  protected final Set<String> validProperties = new HashSet<>();
+  /**
+   * Set of required property names. Subclasses may add to this set
+   * using {@link TriggerUtils#requiredProperties(Set, Set, String...)}
+   * (required properties are also valid properties).
+   */
+  protected final Set<String> requiredProperties = new HashSet<>();
+  /**
+   * Subclasses can add to this set if they want to allow arbitrary properties that
+   * start with one of valid prefixes.
+   */
+  protected final Set<String> validPropertyPrefixes = new HashSet<>();
+
+  protected TriggerListenerBase() {
+    TriggerUtils.requiredProperties(requiredProperties, validProperties, "trigger");
+    TriggerUtils.validProperties(validProperties, "name", "class", "stage", "beforeAction", "afterAction", "enabled");
+  }
 
   @Override
-  public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
+  public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
+    this.loader = loader;
     this.cloudManager = cloudManager;
     this.config = config;
+    this.enabled = Boolean.parseBoolean(String.valueOf(config.properties.getOrDefault("enabled", true)));
+    // validate the config
+    Map<String, String> results = new HashMap<>();
+    // prepare a copy to treat the prefix-based properties
+    Map<String, Object> propsToCheck = new HashMap<>(config.properties);
+    propsToCheck.keySet().removeIf(k ->
+      validPropertyPrefixes.stream().anyMatch(p -> k.startsWith(p)));
+    TriggerUtils.checkProperties(propsToCheck, results, requiredProperties, validProperties);
+    if (!results.isEmpty()) {
+      throw new TriggerValidationException(config.name, results);
+    }
   }
 
   @Override
@@ -41,6 +81,16 @@ public abstract class TriggerListenerBase implements TriggerListener {
   }
 
   @Override
+  public boolean isEnabled() {
+    return enabled;
+  }
+
+  @Override
+  public void init() throws Exception {
+
+  }
+
+  @Override
   public void close() throws IOException {
 
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerUtils.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerUtils.java
new file mode 100644
index 0000000..bdc0c11
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerUtils.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud.autoscaling;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ *
+ */
+public class TriggerUtils {
+  // validation helper methods
+
+  public static void requiredProperties(Set<String> required, Set<String> valid, String... propertyNames) {
+    required.addAll(Arrays.asList(propertyNames));
+    valid.addAll(Arrays.asList(propertyNames));
+  }
+
+  public static void validProperties(Set<String> valid, String... propertyNames) {
+    valid.addAll(Arrays.asList(propertyNames));
+  }
+
+  public static void checkProperties(Map<String, Object> properties, Map<String, String> results, Set<String> required, Set<String> valid) {
+    checkValidPropertyNames(properties, results, valid);
+    checkRequiredPropertyNames(properties, results, required);
+  }
+
+  public static void checkValidPropertyNames(Map<String, Object> properties, Map<String, String> results, Set<String> valid) {
+    Set<String> currentNames = new HashSet<>(properties.keySet());
+    currentNames.removeAll(valid);
+    if (!currentNames.isEmpty()) {
+      for (String name : currentNames) {
+        results.put(name, "unknown property");
+      }
+    }
+  }
+
+  public static void checkRequiredPropertyNames(Map<String, Object> properties, Map<String, String> results, Set<String> required) {
+    Set<String> requiredNames = new HashSet<>(required);
+    requiredNames.removeAll(properties.keySet());
+    if (!requiredNames.isEmpty()) {
+      for (String name : requiredNames) {
+        results.put(name, "missing required property");
+      }
+    }
+  }
+
+  public static void checkProperty(Map<String, Object> properties, Map<String, String> results, String name, boolean required, Class... acceptClasses) {
+    Object value = properties.get(name);
+    if (value == null) {
+      if (required) {
+        results.put(name, "missing required value");
+      } else {
+        return;
+      }
+    }
+    if (acceptClasses == null || acceptClasses.length == 0) {
+      return;
+    }
+    boolean accepted = false;
+    for (Class clz : acceptClasses) {
+      if (clz.isAssignableFrom(value.getClass())) {
+        accepted = true;
+        break;
+      }
+    }
+    if (!accepted) {
+      results.put(name, "value is not an expected type");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerValidationException.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerValidationException.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerValidationException.java
new file mode 100644
index 0000000..648e1e4
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerValidationException.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud.autoscaling;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * This class represents errors found when validating trigger configuration.
+ */
+public class TriggerValidationException extends Exception {
+  private final Map<String, String> details = new HashMap<>();
+  private final String name;
+
+  /**
+   * Create an exception.
+   * @param name name of the trigger / action / listener that caused the exception
+   * @param details details of invalid configuration - key is a property name,
+   *                value is an error message.
+   */
+  public TriggerValidationException(String name, Map<String, String> details) {
+    super();
+    this.name = name;
+    if (details != null) {
+      this.details.putAll(details);
+    }
+  }
+
+  /**
+   * Create an exception.
+   * @param name name of the trigger / action / listener that caused the exception
+   * @param keyValues zero or even number of arguments representing symbolic key
+   *                  (eg. property name) and the corresponding validation error message.
+   */
+  public TriggerValidationException(String name, String... keyValues) {
+    super();
+    this.name = name;
+    if (keyValues == null || keyValues.length == 0) {
+      return;
+    }
+    if (keyValues.length % 2 != 0) {
+      throw new IllegalArgumentException("number of arguments representing key & value pairs must be even");
+    }
+    for (int i = 0; i < keyValues.length; i += 2) {
+      details.put(keyValues[i], keyValues[i + 1]);
+    }
+  }
+
+  public Map<String, String> getDetails() {
+    return details;
+  }
+
+  @Override
+  public String toString() {
+    return "TriggerValidationException{" +
+        "name=" + name +
+        ", details='" + details + '\'' +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/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 78b1274..b98ee70 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
@@ -439,6 +439,123 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
     }
 
   }
+
+  @Test
+  public void testValidation() throws Exception {
+    CloudSolrClient solrClient = cluster.getSolrClient();
+
+    // unknown trigger properties
+    String setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '10m'," +
+        "'enabled' : true," +
+        "'foo': 'bar'," +
+        "'actions' : [" +
+        "{" +
+        "'name' : 'compute_plan'," +
+        "'class' : 'solr.ComputePlanAction'" +
+        "}]}}";
+    SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+
+    try {
+      solrClient.request(req);
+      fail("should have thrown Exception");
+    } catch (HttpSolrClient.RemoteSolrException e) {
+      // expected
+      assertTrue(String.valueOf(getObjectByPath(((HttpSolrClient.RemoteExecutionException) e).getMetaData(),
+          false, "error/details[0]/errorMessages[0]")).contains("foo=unknown property"));
+    }
+
+    // invalid trigger properties
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'search_rate_trigger'," +
+        "'event' : 'searchRate'," +
+        "'waitFor' : '10m'," +
+        "'enabled' : true," +
+        "'rate': 'foo'," +
+        "'actions' : [" +
+        "{" +
+        "'name' : 'compute_plan'," +
+        "'class' : 'solr.ComputePlanAction'" +
+        "}]}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+
+    try {
+      solrClient.request(req);
+      fail("should have thrown Exception");
+    } catch (HttpSolrClient.RemoteSolrException e) {
+      // expected
+      assertTrue(String.valueOf(getObjectByPath(((HttpSolrClient.RemoteExecutionException) e).getMetaData(),
+          false, "error/details[0]/errorMessages[0]")).contains("rate=Invalid 'rate' configuration value: 'foo'"));
+    }
+
+    // unknown trigger action properties
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '10m'," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{" +
+        "'name' : 'compute_plan'," +
+        "'foo' : 'bar'," +
+        "'class' : 'solr.ComputePlanAction'" +
+        "}]}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+
+    try {
+      solrClient.request(req);
+      fail("should have thrown Exception");
+    } catch (HttpSolrClient.RemoteSolrException e) {
+      // expected
+      assertTrue(String.valueOf(getObjectByPath(((HttpSolrClient.RemoteExecutionException) e).getMetaData(),
+          false, "error/details[0]/errorMessages[0]")).contains("foo=unknown property"));
+    }
+
+    // unknown trigger listener properties
+    setTriggerCommand = "{" +
+        "'set-trigger' : {" +
+        "'name' : 'node_lost_trigger'," +
+        "'event' : 'nodeLost'," +
+        "'waitFor' : '10m'," +
+        "'enabled' : true," +
+        "'actions' : [" +
+        "{" +
+        "'name' : 'compute_plan'," +
+        "'class' : 'solr.ComputePlanAction'" +
+        "}]}}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+
+    NamedList<Object> response = solrClient.request(req);
+    assertEquals(response.get("result").toString(), "success");
+
+    String setListenerCommand = "{" +
+        "'set-listener' : " +
+        "{" +
+        "'name' : 'xyz'," +
+        "'trigger' : 'node_lost_trigger'," +
+        "'stage' : ['STARTED','ABORTED','SUCCEEDED']," +
+        "'foo' : 'bar'," +
+        "'beforeAction' : 'execute_plan'," +
+        "'class' : 'org.apache.solr.cloud.autoscaling.HttpTriggerListener'," +
+        "'url' : 'http://xyz.com/on_node_lost?node={$LOST_NODE_NAME}'" +
+        "}" +
+        "}";
+    req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
+    try {
+      solrClient.request(req);
+      fail("should have thrown Exception");
+    } catch (HttpSolrClient.RemoteSolrException e) {
+      // expected
+      assertTrue(String.valueOf(getObjectByPath(((HttpSolrClient.RemoteExecutionException) e).getMetaData(),
+          false, "error/details[0]/errorMessages[0]")).contains("foo=unknown property"));
+    }
+  }
+
   @Test
   public void testPolicyAndPreferences() throws Exception {
     CloudSolrClient solrClient = cluster.getSolrClient();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
index 67b5fa0..4e48f59 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ComputePlanActionTest.java
@@ -47,6 +47,7 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.LogLevel;
 import org.junit.After;
 import org.junit.Before;
@@ -379,6 +380,16 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
     static volatile String expectedNode;
 
     @Override
+    public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
+
+    }
+
+    @Override
+    public void init() {
+
+    }
+
+    @Override
     public String getName() {
       return null;
     }
@@ -400,11 +411,6 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
     public void close() throws IOException {
 
     }
-
-    @Override
-    public void init(Map<String, String> args) {
-
-    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
index 6e4463a..b26da52 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/ExecutePlanActionTest.java
@@ -27,6 +27,7 @@ import java.util.stream.Collectors;
 
 import com.google.common.collect.Lists;
 import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 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;
@@ -40,6 +41,7 @@ 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.Utils;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.LogLevel;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.zookeeper.data.Stat;
@@ -60,6 +62,9 @@ public class ExecutePlanActionTest extends SolrCloudTestCase {
 
   private static final int NODE_COUNT = 2;
 
+  private SolrResourceLoader loader;
+  private SolrCloudManager cloudManager;
+
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(NODE_COUNT)
@@ -81,6 +86,8 @@ public class ExecutePlanActionTest extends SolrCloudTestCase {
       }
     }
     cluster.waitForAllNodes(30);
+    loader = cluster.getJettySolrRunner(0).getCoreContainer().getResourceLoader();
+    cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
   }
 
   @Test
@@ -109,7 +116,7 @@ public class ExecutePlanActionTest extends SolrCloudTestCase {
     JettySolrRunner survivor = otherJetties.get(0);
 
     try (ExecutePlanAction action = new ExecutePlanAction()) {
-      action.init(Collections.singletonMap("name", "execute_plan"));
+      action.configure(loader, cloudManager, Collections.singletonMap("name", "execute_plan"));
 
       // used to signal if we found that ExecutePlanAction did in fact create the right znode before executing the operation
       AtomicBoolean znodeCreated = new AtomicBoolean(false);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/test/org/apache/solr/cloud/autoscaling/HttpTriggerListenerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/HttpTriggerListenerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/HttpTriggerListenerTest.java
index 06fe9a0..eeb1a87 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/HttpTriggerListenerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/HttpTriggerListenerTest.java
@@ -102,10 +102,9 @@ public class HttpTriggerListenerTest extends SolrCloudTestCase {
         "'beforeAction' : 'test'," +
         "'afterAction' : ['test']," +
         "'class' : '" + HttpTriggerListener.class.getName() + "'," +
-        "'url' : '" + mockService.server.getURI().toString() + "/${config.name:invalid}/${config.properties.xyz:invalid}/${stage}'," +
+        "'url' : '" + mockService.server.getURI().toString() + "/${config.name:invalid}/${config.properties.beforeAction:invalid}/${stage}'," +
         "'payload': 'actionName=${actionName}, source=${event.source}, type=${event.eventType}'," +
-        "'header.X-Foo' : '${config.name:invalid}'," +
-        "'xyz': 'foo'" +
+        "'header.X-Foo' : '${config.name:invalid}'" +
         "}" +
         "}";
     req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
@@ -127,19 +126,19 @@ public class HttpTriggerListenerTest extends SolrCloudTestCase {
     requests.forEach(s -> assertTrue(s.contains("type=NODEADDED")));
 
     String request = requests.get(0);
-    assertTrue(request, request.startsWith("/foo/foo/STARTED"));
+    assertTrue(request, request.startsWith("/foo/test/STARTED"));
     assertTrue(request, request.contains("actionName=,")); // empty actionName
 
     request = requests.get(1);
-    assertTrue(request, request.startsWith("/foo/foo/BEFORE_ACTION"));
+    assertTrue(request, request.startsWith("/foo/test/BEFORE_ACTION"));
     assertTrue(request, request.contains("actionName=test,")); // actionName
 
     request = requests.get(2);
-    assertTrue(request, request.startsWith("/foo/foo/AFTER_ACTION"));
+    assertTrue(request, request.startsWith("/foo/test/AFTER_ACTION"));
     assertTrue(request, request.contains("actionName=test,")); // actionName
 
     request = requests.get(3);
-    assertTrue(request, request.startsWith("/foo/foo/SUCCEEDED"));
+    assertTrue(request, request.startsWith("/foo/test/SUCCEEDED"));
     assertTrue(request, request.contains("actionName=,")); // empty actionName
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerIntegrationTest.java
index 7b6da5a..67d943f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerIntegrationTest.java
@@ -40,6 +40,7 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.util.LogLevel;
 import org.junit.BeforeClass;
@@ -227,8 +228,8 @@ public class MetricTriggerIntegrationTest extends SolrCloudTestCase {
 
   public static class TestTriggerListener extends TriggerListenerBase {
     @Override
-    public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
-      super.init(cloudManager, config);
+    public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
+      super.configure(loader, cloudManager, config);
       listenerCreated.countDown();
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerTest.java
index 93c91c5..f0f9f07 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/MetricTriggerTest.java
@@ -72,7 +72,8 @@ public class MetricTriggerTest extends SolrCloudTestCase {
     SolrResourceLoader loader = cluster.getJettySolrRunner(0).getCoreContainer().getResourceLoader();
     SolrCloudManager cloudManager = new SolrClientCloudManager(new ZkDistributedQueueFactory(zkClient), cluster.getSolrClient());
 
-    try (MetricTrigger metricTrigger = new MetricTrigger("metricTrigger", props, loader, cloudManager)) {
+    try (MetricTrigger metricTrigger = new MetricTrigger("metricTrigger")) {
+      metricTrigger.configure(loader, cloudManager, props);
       metricTrigger.setProcessor(noFirstRunProcessor);
       metricTrigger.run();
       metricTrigger.setProcessor(event -> events.add(event));
@@ -85,7 +86,8 @@ public class MetricTriggerTest extends SolrCloudTestCase {
     events.clear();
     tag = "metrics:" + registry + ":ADMIN./admin/file.handlerStart";
     props = createTriggerProps(waitForSeconds, tag, null, 100.0d, DEFAULT_TEST_COLLECTION_NAME, null, null);
-    try (MetricTrigger metricTrigger = new MetricTrigger("metricTrigger", props, loader, cloudManager)) {
+    try (MetricTrigger metricTrigger = new MetricTrigger("metricTrigger")) {
+      metricTrigger.configure(loader, cloudManager, props);
       metricTrigger.setProcessor(noFirstRunProcessor);
       metricTrigger.run();
       metricTrigger.setProcessor(event -> events.add(event));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
index ecf2437..ddc56ec 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerIntegrationTest.java
@@ -19,7 +19,6 @@ package org.apache.solr.cloud.autoscaling;
 
 import java.lang.invoke.MethodHandles;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
@@ -291,10 +290,10 @@ public class NodeAddedTriggerIntegrationTest extends SolrCloudTestCase {
     }
 
     @Override
-    public void init(Map<String, String> args) {
+    public void init() throws Exception {
       log.info("TestTriggerAction init");
       actionInitCalled.countDown();
-      super.init(args);
+      super.init();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a14980c4/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
index d82cec2..2d084b8 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeAddedTriggerTest.java
@@ -27,10 +27,12 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.SolrResourceLoader;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -44,7 +46,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
   private static AtomicBoolean actionCloseCalled = new AtomicBoolean(false);
 
   private AutoScaling.TriggerEventProcessor noFirstRunProcessor = event -> {
-    fail("Did not expect the listener to fire on first run!");
+    fail("Did not expect the processor to fire on first run! event=" + event);
     return true;
   };
 
@@ -72,8 +74,9 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
     long waitForSeconds = 1 + random().nextInt(5);
     Map<String, Object> props = createTriggerProps(waitForSeconds);
 
-    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(),
-        container.getZkController().getSolrCloudManager())) {
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger")) {
+      trigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
+      trigger.init();
       trigger.setProcessor(noFirstRunProcessor);
       trigger.run();
 
@@ -113,8 +116,9 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
 
     // add a new node but remove it before the waitFor period expires
     // and assert that the trigger doesn't fire at all
-    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(),
-        container.getZkController().getSolrCloudManager())) {
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger")) {
+      trigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
+      trigger.init();
       final long waitTime = 2;
       props.put("waitFor", waitTime);
       trigger.setProcessor(noFirstRunProcessor);
@@ -159,8 +163,8 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
     action.put("name", "testActionInit");
     action.put("class", NodeAddedTriggerTest.AssertInitTriggerAction.class.getName());
     actions.add(action);
-    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(),
-        container.getZkController().getSolrCloudManager())) {
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger")) {
+      trigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
       assertEquals(true, actionConstructorCalled.get());
       assertEquals(false, actionInitCalled.get());
       assertEquals(false, actionCloseCalled.get());
@@ -177,6 +181,16 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
     }
 
     @Override
+    public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
+
+    }
+
+    @Override
+    public void init() {
+      actionInitCalled.compareAndSet(false, true);
+    }
+
+    @Override
     public String getName() {
       return "";
     }
@@ -190,19 +204,15 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
     public void close() throws IOException {
       actionCloseCalled.compareAndSet(false, true);
     }
-
-    @Override
-    public void init(Map<String, String> args) {
-      actionInitCalled.compareAndSet(false, true);
-    }
   }
 
   @Test
   public void testListenerAcceptance() throws Exception {
     CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
     Map<String, Object> props = createTriggerProps(0);
-    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(),
-        container.getZkController().getSolrCloudManager())) {
+    try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger")) {
+      trigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
+      trigger.init();
       trigger.setProcessor(noFirstRunProcessor);
       trigger.run(); // starts tracking live nodes
 
@@ -238,8 +248,8 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
 
     // add a new node but update the trigger before the waitFor period expires
     // and assert that the new trigger still fires
-    NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(),
-        container.getZkController().getSolrCloudManager());
+    NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger");
+    trigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
     trigger.setProcessor(noFirstRunProcessor);
     trigger.run();
 
@@ -247,8 +257,9 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
     trigger.run(); // this run should detect the new node
     trigger.close(); // close the old trigger
 
-    try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("some_different_name", props, container.getResourceLoader(),
-        container.getZkController().getSolrCloudManager()))  {
+    try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("some_different_name"))  {
+      newTrigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
+      newTrigger.init();
       try {
         newTrigger.restoreState(trigger);
         fail("Trigger should only be able to restore state from an old trigger of the same name");
@@ -257,8 +268,9 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
       }
     }
 
-    try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(),
-        container.getZkController().getSolrCloudManager()))  {
+    try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("node_added_trigger"))  {
+      newTrigger.configure(container.getResourceLoader(), container.getZkController().getSolrCloudManager(), props);
+      newTrigger.init();
       AtomicBoolean fired = new AtomicBoolean(false);
       AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
       newTrigger.setProcessor(event -> {