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 2017/05/24 09:11:18 UTC

lucene-solr:jira/solr-10515: SOLR-10515 Collapse TriggerEventBase -> TriggerEvent, remove unnecessary generics.

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-10515 39464e2cb -> 15d637fe0


SOLR-10515 Collapse TriggerEventBase -> TriggerEvent, remove unnecessary generics.


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

Branch: refs/heads/jira/solr-10515
Commit: 15d637fe066acec92e631c2f602172f9ea5ef9f3
Parents: 39464e2
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Wed May 24 11:09:57 2017 +0200
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Wed May 24 11:09:57 2017 +0200

----------------------------------------------------------------------
 .../solr/cloud/autoscaling/AutoScaling.java     |  49 +-----
 .../cloud/autoscaling/ComputePlanAction.java    |   2 +-
 .../cloud/autoscaling/ExecutePlanAction.java    |   2 +-
 .../solr/cloud/autoscaling/LogPlanAction.java   |   2 +-
 .../cloud/autoscaling/NodeAddedTrigger.java     |  14 +-
 .../solr/cloud/autoscaling/NodeLostTrigger.java |  14 +-
 .../cloud/autoscaling/ScheduledTriggers.java    |  19 +--
 .../solr/cloud/autoscaling/TriggerAction.java   |   2 +-
 .../solr/cloud/autoscaling/TriggerBase.java     |   2 +-
 .../solr/cloud/autoscaling/TriggerEvent.java    | 153 +++++++++++++++++++
 .../cloud/autoscaling/TriggerEventBase.java     | 134 ----------------
 .../cloud/autoscaling/TriggerEventQueue.java    |  21 ++-
 .../cloud/autoscaling/NodeAddedTriggerTest.java |  10 +-
 .../cloud/autoscaling/NodeLostTriggerTest.java  |  10 +-
 .../autoscaling/TriggerIntegrationTest.java     |  15 +-
 15 files changed, 215 insertions(+), 234 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/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 4beffd8..cf6a1e4 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
@@ -19,13 +19,11 @@ package org.apache.solr.cloud.autoscaling;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import com.google.common.base.Preconditions;
 import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.solr.common.MapWriter;
 import org.apache.solr.core.CoreContainer;
 
 public class AutoScaling {
@@ -50,38 +48,7 @@ public class AutoScaling {
     AFTER_ACTION
   }
 
-  public interface TriggerEvent extends MapWriter {
-    /** Unique event id. */
-    String getId();
-
-    /** Event type. */
-    EventType getEventType();
-
-    /** Name of the trigger that fired the event. */
-    String getSource();
-
-    /** Timestamp of the actual event.
-     * NOTE: this is NOT the timestamp when the event was fired - events may be fired
-     * much later than the actual condition that generated the event, due to the "waitFor" limit.
-     */
-    long getEventTime();
-
-    /**
-     * Set event properties.
-     * @param properties may be null. A shallow copy of this parameter is used.
-     */
-    void setProperties(Map<String, Object> properties);
-
-    /**
-     * Get event properties (modifiable).
-     */
-    Map<String, Object> getProperties();
-
-    /** Get a named event property or null if missing. */
-    Object getProperty(String name);
-  }
-
-  public interface TriggerListener<E extends TriggerEvent> {
+  public interface TriggerListener {
     /**
      * This method is executed when a trigger is ready to fire.
      *
@@ -89,7 +56,7 @@ public class AutoScaling {
      * @return true if the listener was ready to perform actions on the event, false
      * otherwise. If false was returned then callers should assume the event was discarded.
      */
-    boolean triggerFired(E event);
+    boolean triggerFired(TriggerEvent event);
   }
 
   public static class HttpCallbackListener implements TriggerListener {
@@ -102,7 +69,7 @@ public class AutoScaling {
   /**
    * Interface for a Solr trigger. Each trigger implements Runnable and Closeable interface. A trigger
    * is scheduled using a {@link java.util.concurrent.ScheduledExecutorService} so it is executed as
-   * per a configured schedule to check whether the trigger is ready to fire. The {@link #setListener(TriggerListener)}
+   * per a configured schedule to check whether the trigger is ready to fire. The {@link Trigger#setListener(TriggerListener)}
    * method should be used to set a callback listener which is fired by implementation of this class whenever
    * ready.
    * <p>
@@ -116,10 +83,8 @@ public class AutoScaling {
    * with the proper trigger event object. If that method returns false then it should be interpreted to mean
    * that Solr is not ready to process this trigger event and therefore we should retain the state and fire
    * at the next invocation of the run() method.
-   *
-   * @param <E> the {@link TriggerEvent} which is handled by this Trigger
    */
-  public interface Trigger<E extends TriggerEvent> extends Closeable, Runnable {
+  public interface Trigger extends Closeable, Runnable {
     /**
      * Trigger name.
      */
@@ -143,16 +108,16 @@ public class AutoScaling {
     List<TriggerAction> getActions();
 
     /** Set event listener to call when event is fired. */
-    void setListener(TriggerListener<E> listener);
+    void setListener(TriggerListener listener);
 
     /** Get event listener. */
-    TriggerListener<E> getListener();
+    TriggerListener getListener();
 
     /** Return true when this trigger is closed and cannot be used. */
     boolean isClosed();
 
     /** Set internal state of this trigger from another instance. */
-    void restoreState(Trigger<E> old);
+    void restoreState(Trigger old);
 
     /** Save internal state of this trigger in ZooKeeper. */
     void saveState();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/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 3e7aff6..1b8e680 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
@@ -45,7 +45,7 @@ public class ComputePlanAction implements TriggerAction {
   }
 
   @Override
-  public void process(AutoScaling.TriggerEvent event) {
+  public void process(TriggerEvent event) {
 
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
index 5950948..90a7cf7 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ExecutePlanAction.java
@@ -45,7 +45,7 @@ public class ExecutePlanAction implements TriggerAction {
   }
 
   @Override
-  public void process(AutoScaling.TriggerEvent event) {
+  public void process(TriggerEvent event) {
 
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/solr/core/src/java/org/apache/solr/cloud/autoscaling/LogPlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/LogPlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/LogPlanAction.java
index fc86c96..f89e8d9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/LogPlanAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/LogPlanAction.java
@@ -45,7 +45,7 @@ public class LogPlanAction implements TriggerAction {
   }
 
   @Override
-  public void process(AutoScaling.TriggerEvent event) {
+  public void process(TriggerEvent event) {
 
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/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 c3bbffe..c93b6b9 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
@@ -44,14 +44,14 @@ import org.slf4j.LoggerFactory;
 /**
  * Trigger for the {@link org.apache.solr.cloud.autoscaling.AutoScaling.EventType#NODEADDED} event
  */
-public class NodeAddedTrigger extends TriggerBase<NodeAddedTrigger.NodeAddedEvent> {
+public class NodeAddedTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private final String name;
   private final Map<String, Object> properties;
   private final CoreContainer container;
   private final List<TriggerAction> actions;
-  private final AtomicReference<AutoScaling.TriggerListener<NodeAddedEvent>> listenerRef;
+  private final AtomicReference<AutoScaling.TriggerListener> listenerRef;
   private final boolean enabled;
   private final int waitForSecond;
   private final AutoScaling.EventType eventType;
@@ -89,12 +89,12 @@ public class NodeAddedTrigger extends TriggerBase<NodeAddedTrigger.NodeAddedEven
   }
 
   @Override
-  public void setListener(AutoScaling.TriggerListener<NodeAddedEvent> listener) {
+  public void setListener(AutoScaling.TriggerListener listener) {
     listenerRef.set(listener);
   }
 
   @Override
-  public AutoScaling.TriggerListener<NodeAddedEvent> getListener() {
+  public AutoScaling.TriggerListener getListener() {
     return listenerRef.get();
   }
 
@@ -152,7 +152,7 @@ public class NodeAddedTrigger extends TriggerBase<NodeAddedTrigger.NodeAddedEven
   }
 
   @Override
-  public void restoreState(AutoScaling.Trigger<NodeAddedEvent> old) {
+  public void restoreState(AutoScaling.Trigger old) {
     assert old.isClosed();
     if (old instanceof NodeAddedTrigger) {
       NodeAddedTrigger that = (NodeAddedTrigger) old;
@@ -223,7 +223,7 @@ public class NodeAddedTrigger extends TriggerBase<NodeAddedTrigger.NodeAddedEven
         long now = System.currentTimeMillis();
         if (TimeUnit.SECONDS.convert(now - timeAdded, TimeUnit.MILLISECONDS) >= getWaitForSecond()) {
           // fire!
-          AutoScaling.TriggerListener<NodeAddedEvent> listener = listenerRef.get();
+          AutoScaling.TriggerListener listener = listenerRef.get();
           if (listener != null) {
             log.debug("NodeAddedTrigger {} firing registered listener for node: {} added at time {} , now: {}", name, nodeName, timeAdded, now);
             if (listener.triggerFired(new NodeAddedEvent(getEventType(), getName(), timeAdded, nodeName))) {
@@ -249,7 +249,7 @@ public class NodeAddedTrigger extends TriggerBase<NodeAddedTrigger.NodeAddedEven
     }
   }
 
-  public static class NodeAddedEvent extends TriggerEventBase {
+  public static class NodeAddedEvent extends TriggerEvent {
 
     public NodeAddedEvent(AutoScaling.EventType eventType, String source, long nodeAddedTime, String nodeAdded) {
       super(eventType, source, nodeAddedTime, Collections.singletonMap(NODE_NAME, nodeAdded));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/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 feea94d..aa9110c 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
@@ -44,14 +44,14 @@ import org.slf4j.LoggerFactory;
 /**
  * Trigger for the {@link AutoScaling.EventType#NODELOST} event
  */
-public class NodeLostTrigger extends TriggerBase<NodeLostTrigger.NodeLostEvent> {
+public class NodeLostTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private final String name;
   private final Map<String, Object> properties;
   private final CoreContainer container;
   private final List<TriggerAction> actions;
-  private final AtomicReference<AutoScaling.TriggerListener<NodeLostEvent>> listenerRef;
+  private final AtomicReference<AutoScaling.TriggerListener> listenerRef;
   private final boolean enabled;
   private final int waitForSecond;
   private final AutoScaling.EventType eventType;
@@ -88,12 +88,12 @@ public class NodeLostTrigger extends TriggerBase<NodeLostTrigger.NodeLostEvent>
   }
 
   @Override
-  public void setListener(AutoScaling.TriggerListener<NodeLostEvent> listener) {
+  public void setListener(AutoScaling.TriggerListener listener) {
     listenerRef.set(listener);
   }
 
   @Override
-  public AutoScaling.TriggerListener<NodeLostEvent> getListener() {
+  public AutoScaling.TriggerListener getListener() {
     return listenerRef.get();
   }
 
@@ -151,7 +151,7 @@ public class NodeLostTrigger extends TriggerBase<NodeLostTrigger.NodeLostEvent>
   }
 
   @Override
-  public void restoreState(AutoScaling.Trigger<NodeLostEvent> old) {
+  public void restoreState(AutoScaling.Trigger old) {
     assert old.isClosed();
     if (old instanceof NodeLostTrigger) {
       NodeLostTrigger that = (NodeLostTrigger) old;
@@ -220,7 +220,7 @@ public class NodeLostTrigger extends TriggerBase<NodeLostTrigger.NodeLostEvent>
         Long timeRemoved = entry.getValue();
         if (TimeUnit.SECONDS.convert(System.currentTimeMillis() - timeRemoved, TimeUnit.MILLISECONDS) >= getWaitForSecond()) {
           // fire!
-          AutoScaling.TriggerListener<NodeLostEvent> listener = listenerRef.get();
+          AutoScaling.TriggerListener listener = listenerRef.get();
           if (listener != null) {
             log.debug("NodeLostTrigger firing registered listener");
             if (listener.triggerFired(new NodeLostEvent(getEventType(), getName(), timeRemoved, nodeName)))  {
@@ -245,7 +245,7 @@ public class NodeLostTrigger extends TriggerBase<NodeLostTrigger.NodeLostEvent>
     }
   }
 
-  public static class NodeLostEvent extends TriggerEventBase {
+  public static class NodeLostEvent extends TriggerEvent {
 
     public NodeLostEvent(AutoScaling.EventType eventType, String source, long nodeLostTime, String nodeRemoved) {
       super(eventType, source, nodeLostTime, Collections.singletonMap(NODE_NAME, nodeRemoved));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/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 3e94d1c..52e3d46 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
@@ -21,7 +21,6 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -36,10 +35,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.solr.cloud.ActionThrottle;
-import org.apache.solr.cloud.DistributedQueue;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.util.DefaultSolrThreadFactory;
@@ -125,7 +122,7 @@ public class ScheduledTriggers implements Closeable {
         log.warn("Ignoring autoscaling event " + event + " because the source trigger: " + event.getSource() + " doesn't exist.");
         return false;
       }
-      boolean replaying = event.getProperty(TriggerEventBase.REPLAYING) != null ? (Boolean)event.getProperty(TriggerEventBase.REPLAYING) : false;
+      boolean replaying = event.getProperty(TriggerEvent.REPLAYING) != null ? (Boolean)event.getProperty(TriggerEvent.REPLAYING) : false;
       AutoScaling.Trigger source = scheduledSource.trigger;
       if (source.isClosed()) {
         log.warn("Ignoring autoscaling event " + event + " because the source trigger: " + source + " has already been closed");
@@ -157,7 +154,7 @@ public class ScheduledTriggers implements Closeable {
                 }
               }
               if (enqueued) {
-                AutoScaling.TriggerEvent ev = scheduledTrigger.dequeue();
+                TriggerEvent ev = scheduledTrigger.dequeue();
                 assert ev.getId().equals(event.getId());
               }
             } finally {
@@ -166,7 +163,7 @@ public class ScheduledTriggers implements Closeable {
           });
         } else {
           if (enqueued) {
-            AutoScaling.TriggerEvent ev = scheduledTrigger.dequeue();
+            TriggerEvent ev = scheduledTrigger.dequeue();
             assert ev.getId().equals(event.getId());
           }
           hasPendingActions.set(false);
@@ -227,7 +224,7 @@ public class ScheduledTriggers implements Closeable {
       this.isClosed = false;
     }
 
-    public boolean enqueue(AutoScaling.TriggerEvent event) {
+    public boolean enqueue(TriggerEvent event) {
       if (isClosed) {
         throw new AlreadyClosedException("ScheduledTrigger " + trigger.getName() + " has been closed.");
       }
@@ -235,11 +232,11 @@ public class ScheduledTriggers implements Closeable {
       return queue.offerEvent(event);
     }
 
-    public AutoScaling.TriggerEvent dequeue() {
+    public TriggerEvent dequeue() {
       if (isClosed) {
         throw new AlreadyClosedException("ScheduledTrigger " + trigger.getName() + " has been closed.");
       }
-      AutoScaling.TriggerEvent event = queue.pollEvent();
+      TriggerEvent event = queue.pollEvent();
       log.debug("--Dequeue " + event);
       return event;
     }
@@ -253,11 +250,11 @@ public class ScheduledTriggers implements Closeable {
       // replay accumulated events on first run, if any
       if (replay) {
         log.debug(" --replaying...");
-        AutoScaling.TriggerEvent event;
+        TriggerEvent event;
         // peek first without removing - we may crash before calling the listener
         while ((event = queue.peekEvent()) != null) {
           // override REPLAYING=true
-          event.getProperties().put(TriggerEventBase.REPLAYING, true);
+          event.getProperties().put(TriggerEvent.REPLAYING, true);
           if (! trigger.getListener().triggerFired(event)) {
             log.error("Failed to re-play event, discarding: " + event);
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/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 242c9de..b00dfd0 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
@@ -30,5 +30,5 @@ public interface TriggerAction extends MapInitializedPlugin, Closeable {
 
   public String getClassName();
 
-  public void process(AutoScaling.TriggerEvent event);
+  public void process(TriggerEvent event);
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/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 235cc9d..89b3d38 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
@@ -34,7 +34,7 @@ import org.slf4j.LoggerFactory;
  * Base class for {@link org.apache.solr.cloud.autoscaling.AutoScaling.Trigger} implementations.
  * It handles state snapshot / restore in ZK.
  */
-public abstract class TriggerBase<E extends AutoScaling.TriggerEvent> implements AutoScaling.Trigger<E> {
+public abstract class TriggerBase implements AutoScaling.Trigger {
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   protected SolrZkClient zkClient;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java
new file mode 100644
index 0000000..f14988a
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEvent.java
@@ -0,0 +1,153 @@
+/*
+ * 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.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.util.IdUtils;
+
+/**
+ * Trigger event.
+ */
+public class TriggerEvent implements MapWriter {
+  public static final String REPLAYING = "replaying";
+  public static final String NODE_NAME = "nodeName";
+
+  protected final String id;
+  protected final String source;
+  protected final long eventTime;
+  protected final AutoScaling.EventType eventType;
+  protected final Map<String, Object> properties = new HashMap<>();
+
+  public TriggerEvent(AutoScaling.EventType eventType, String source, long eventTime,
+                      Map<String, Object> properties) {
+    this(IdUtils.timeRandomId(eventTime), eventType, source, eventTime, properties);
+  }
+
+  public TriggerEvent(String id, AutoScaling.EventType eventType, String source, long eventTime,
+                      Map<String, Object> properties) {
+    this.id = id;
+    this.eventType = eventType;
+    this.source = source;
+    this.eventTime = eventTime;
+    if (properties != null) {
+      this.properties.putAll(properties);
+    }
+  }
+
+  /**
+   * Unique event id.
+   */
+  public String getId() {
+    return id;
+  }
+
+  /**
+   * Name of the trigger that fired the event.
+   */
+  public String getSource() {
+    return source;
+  }
+
+  /**
+   * Timestamp of the actual event.
+   * NOTE: this is NOT the timestamp when the event was fired - events may be fired
+   * much later than the actual condition that generated the event, due to the "waitFor" limit.
+   */
+  public long getEventTime() {
+    return eventTime;
+  }
+
+  /**
+   * Get event properties (modifiable).
+   */
+  public Map<String, Object> getProperties() {
+    return properties;
+  }
+
+  /**
+   * Get a named event property or null if missing.
+   */
+  public Object getProperty(String name) {
+    return properties.get(name);
+  }
+
+  /**
+   * Event type.
+   */
+  public AutoScaling.EventType getEventType() {
+    return eventType;
+  }
+
+  /**
+   * Set event properties.
+   *
+   * @param properties may be null. A shallow copy of this parameter is used.
+   */
+  public void setProperties(Map<String, Object> properties) {
+    this.properties.clear();
+    if (properties != null) {
+      this.properties.putAll(properties);
+    }
+  }
+
+  @Override
+  public void writeMap(EntryWriter ew) throws IOException {
+    ew.put("id", id);
+    ew.put("source", source);
+    ew.put("eventTime", eventTime);
+    ew.put("eventType", eventType.toString());
+    ew.put("properties", properties);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    TriggerEvent that = (TriggerEvent) o;
+
+    if (eventTime != that.eventTime) return false;
+    if (!id.equals(that.id)) return false;
+    if (!source.equals(that.source)) return false;
+    if (eventType != that.eventType) return false;
+    return properties.equals(that.properties);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = id.hashCode();
+    result = 31 * result + source.hashCode();
+    result = 31 * result + (int) (eventTime ^ (eventTime >>> 32));
+    result = 31 * result + eventType.hashCode();
+    result = 31 * result + properties.hashCode();
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return this.getClass().getSimpleName() + "{" +
+        "id='" + id + '\'' +
+        ", source='" + source + '\'' +
+        ", eventTime=" + eventTime +
+        ", properties=" + properties +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventBase.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventBase.java
deleted file mode 100644
index 13fd611..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventBase.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * 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.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.solr.util.IdUtils;
-
-/**
- * Base class for event implementations.
- */
-public abstract class TriggerEventBase implements AutoScaling.TriggerEvent {
-  public static final String REPLAYING = "replaying";
-  public static final String NODE_NAME = "nodeName";
-
-  protected final String id;
-  protected final String source;
-  protected final long eventTime;
-  protected final AutoScaling.EventType eventType;
-  protected final Map<String, Object> properties = new HashMap<>();
-
-  protected TriggerEventBase(AutoScaling.EventType eventType, String source, long eventTime,
-                             Map<String, Object> properties) {
-    this(IdUtils.timeRandomId(eventTime), eventType, source, eventTime, properties);
-  }
-
-  protected TriggerEventBase(String id, AutoScaling.EventType eventType, String source, long eventTime,
-                             Map<String, Object> properties) {
-    this.id = id;
-    this.eventType = eventType;
-    this.source = source;
-    this.eventTime = eventTime;
-    if (properties != null) {
-      this.properties.putAll(properties);
-    }
-  }
-
-  @Override
-  public String getId() {
-    return id;
-  }
-
-  @Override
-  public String getSource() {
-    return source;
-  }
-
-  @Override
-  public long getEventTime() {
-    return eventTime;
-  }
-
-  @Override
-  public Map<String, Object> getProperties() {
-    return properties;
-  }
-
-  @Override
-  public Object getProperty(String name) {
-    return properties.get(name);
-  }
-
-  @Override
-  public AutoScaling.EventType getEventType() {
-    return eventType;
-  }
-
-  @Override
-  public void setProperties(Map<String, Object> context) {
-    this.properties.clear();
-    if (context != null) {
-      this.properties.putAll(context);
-    }
-  }
-
-  @Override
-  public void writeMap(EntryWriter ew) throws IOException {
-    ew.put("id", id);
-    ew.put("source", source);
-    ew.put("eventTime", eventTime);
-    ew.put("eventType", eventType.toString());
-    ew.put("properties", properties);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    TriggerEventBase that = (TriggerEventBase) o;
-
-    if (eventTime != that.eventTime) return false;
-    if (!id.equals(that.id)) return false;
-    if (!source.equals(that.source)) return false;
-    if (eventType != that.eventType) return false;
-    return properties.equals(that.properties);
-  }
-
-  @Override
-  public int hashCode() {
-    int result = id.hashCode();
-    result = 31 * result + source.hashCode();
-    result = 31 * result + (int) (eventTime ^ (eventTime >>> 32));
-    result = 31 * result + eventType.hashCode();
-    result = 31 * result + properties.hashCode();
-    return result;
-  }
-
-  @Override
-  public String toString() {
-    return this.getClass().getSimpleName() + "{" +
-        "id='" + id + '\'' +
-        ", source='" + source + '\'' +
-        ", eventTime=" + eventTime +
-        ", properties=" + properties +
-        '}';
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
index df1b2c4..19427f5 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerEventQueue.java
@@ -18,12 +18,8 @@ import org.slf4j.LoggerFactory;
 public class TriggerEventQueue extends DistributedQueue {
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static class QueuedEvent extends TriggerEventBase {
-
-    public QueuedEvent(String id, AutoScaling.EventType eventType, String source, long eventTime, Map<String, Object> properties) {
-      super(id, eventType, source, eventTime, properties);
-    }
-  }
+  public static final String ENQUEUE_TIME = "_enqueue_time_";
+  public static final String DEQUEUE_TIME = "_dequeue_time_";
 
   private final String triggerName;
 
@@ -32,7 +28,8 @@ public class TriggerEventQueue extends DistributedQueue {
     this.triggerName = triggerName;
   }
 
-  public boolean offerEvent(AutoScaling.TriggerEvent event) {
+  public boolean offerEvent(TriggerEvent event) {
+    event.getProperties().put(ENQUEUE_TIME, System.currentTimeMillis());
     try {
       byte[] data = Utils.toJSON(event);
       offer(data);
@@ -43,7 +40,7 @@ public class TriggerEventQueue extends DistributedQueue {
     }
   }
 
-  public AutoScaling.TriggerEvent peekEvent() {
+  public TriggerEvent peekEvent() {
     byte[] data;
     try {
       while ((data = peek()) != null) {
@@ -65,7 +62,7 @@ public class TriggerEventQueue extends DistributedQueue {
     return null;
   }
 
-  public AutoScaling.TriggerEvent pollEvent() {
+  public TriggerEvent pollEvent() {
     byte[] data;
     try {
       while ((data = poll()) != null) {
@@ -87,12 +84,14 @@ public class TriggerEventQueue extends DistributedQueue {
     return null;
   }
 
-  private static QueuedEvent fromMap(Map<String, Object> map) {
+  private static TriggerEvent fromMap(Map<String, Object> map) {
     String id = (String)map.get("id");
     String source = (String)map.get("source");
     long eventTime = ((Number)map.get("eventTime")).longValue();
     AutoScaling.EventType eventType = AutoScaling.EventType.valueOf((String)map.get("eventType"));
     Map<String, Object> properties = (Map<String, Object>)map.get("properties");
-    return new QueuedEvent(id, eventType, source, eventTime, properties);
+    TriggerEvent res = new TriggerEvent(id, eventType, source, eventTime, properties);
+    res.getProperties().put(DEQUEUE_TIME, System.currentTimeMillis());
+    return res;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/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 ef8ccf1..68c0d0c 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
@@ -37,7 +37,7 @@ import org.junit.Test;
  */
 public class NodeAddedTriggerTest extends SolrCloudTestCase {
 
-  private AutoScaling.TriggerListener<NodeAddedTrigger.NodeAddedEvent> noFirstRunListener = event -> {
+  private AutoScaling.TriggerListener noFirstRunListener = event -> {
     fail("Did not expect the listener to fire on first run!");
     return true;
   };
@@ -61,7 +61,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
 
       JettySolrRunner newNode = cluster.startJettySolrRunner();
       AtomicBoolean fired = new AtomicBoolean(false);
-      AtomicReference<NodeAddedTrigger.NodeAddedEvent> eventRef = new AtomicReference<>();
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
       trigger.setListener(event -> {
         if (fired.compareAndSet(false, true)) {
           eventRef.set(event);
@@ -82,7 +82,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
         }
       } while (!fired.get());
 
-      NodeAddedTrigger.NodeAddedEvent nodeAddedEvent = eventRef.get();
+      TriggerEvent nodeAddedEvent = eventRef.get();
       assertNotNull(nodeAddedEvent);
       assertEquals("", newNode.getNodeName(), nodeAddedEvent.getProperty(NodeAddedTrigger.NodeAddedEvent.NODE_NAME));
     }
@@ -184,7 +184,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
 
     try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("node_added_trigger", props, container))  {
       AtomicBoolean fired = new AtomicBoolean(false);
-      AtomicReference<NodeAddedTrigger.NodeAddedEvent> eventRef = new AtomicReference<>();
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
       newTrigger.setListener(event -> {
         if (fired.compareAndSet(false, true)) {
           eventRef.set(event);
@@ -208,7 +208,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
 
       // ensure the event was fired
       assertTrue(fired.get());
-      NodeAddedTrigger.NodeAddedEvent nodeAddedEvent = eventRef.get();
+      TriggerEvent nodeAddedEvent = eventRef.get();
       assertNotNull(nodeAddedEvent);
       assertEquals("", newNode.getNodeName(), nodeAddedEvent.getProperty(NodeAddedTrigger.NodeAddedEvent.NODE_NAME));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
index ec110dc..e916ab6 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeLostTriggerTest.java
@@ -37,7 +37,7 @@ import org.junit.Test;
  */
 public class NodeLostTriggerTest extends SolrCloudTestCase {
 
-  private AutoScaling.TriggerListener<NodeLostTrigger.NodeLostEvent> noFirstRunListener = event -> {
+  private AutoScaling.TriggerListener noFirstRunListener = event -> {
     fail("Did not expect the listener to fire on first run!");
     return true;
   };
@@ -62,7 +62,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
       cluster.stopJettySolrRunner(1);
 
       AtomicBoolean fired = new AtomicBoolean(false);
-      AtomicReference<NodeLostTrigger.NodeLostEvent> eventRef = new AtomicReference<>();
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
       trigger.setListener(event -> {
         if (fired.compareAndSet(false, true)) {
           eventRef.set(event);
@@ -83,7 +83,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
         }
       } while (!fired.get());
 
-      NodeLostTrigger.NodeLostEvent nodeLostEvent = eventRef.get();
+      TriggerEvent nodeLostEvent = eventRef.get();
       assertNotNull(nodeLostEvent);
       assertEquals("", lostNodeName, nodeLostEvent.getProperty(NodeLostTrigger.NodeLostEvent.NODE_NAME));
 
@@ -211,7 +211,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
 
     try (NodeLostTrigger newTrigger = new NodeLostTrigger("node_lost_trigger", props, container)) {
       AtomicBoolean fired = new AtomicBoolean(false);
-      AtomicReference<NodeLostTrigger.NodeLostEvent> eventRef = new AtomicReference<>();
+      AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
       newTrigger.setListener(event -> {
         if (fired.compareAndSet(false, true)) {
           eventRef.set(event);
@@ -233,7 +233,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
         }
       } while (!fired.get());
 
-      NodeLostTrigger.NodeLostEvent nodeLostEvent = eventRef.get();
+      TriggerEvent nodeLostEvent = eventRef.get();
       assertNotNull(nodeLostEvent);
       assertEquals("", lostNodeName, nodeLostEvent.getProperty(NodeLostTrigger.NodeLostEvent.NODE_NAME));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/15d637fe/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
index ac22500..c43fabf 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
@@ -65,7 +65,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
   private static CyclicBarrier actionInterrupted;
   private static CyclicBarrier actionCompleted;
   private static AtomicBoolean triggerFired;
-  private static AtomicReference<AutoScaling.TriggerEvent> eventRef;
+  private static AtomicReference<TriggerEvent> eventRef;
 
   private String path;
 
@@ -204,7 +204,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     private final AtomicBoolean onlyOnce = new AtomicBoolean(false);
 
     @Override
-    public void process(AutoScaling.TriggerEvent event) {
+    public void process(TriggerEvent event) {
       boolean locked = lock.tryLock();
       if (!locked)  {
         log.info("We should never have a tryLock fail because actions are never supposed to be executed concurrently");
@@ -473,7 +473,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     NodeAddedTrigger.NodeAddedEvent nodeAddedEvent = (NodeAddedTrigger.NodeAddedEvent) eventRef.get();
     assertNotNull(nodeAddedEvent);
     assertEquals("The node added trigger was fired but for a different node",
-        newNode.getNodeName(), nodeAddedEvent.getProperty(TriggerEventBase.NODE_NAME));
+        newNode.getNodeName(), nodeAddedEvent.getProperty(TriggerEvent.NODE_NAME));
   }
 
   public static class TestTriggerAction implements TriggerAction {
@@ -494,7 +494,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     }
 
     @Override
-    public void process(AutoScaling.TriggerEvent event) {
+    public void process(TriggerEvent event) {
       try {
         if (triggerFired.compareAndSet(false, true))  {
           eventRef.set(event);
@@ -540,7 +540,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     }
 
     @Override
-    public void process(AutoScaling.TriggerEvent event) {
+    public void process(TriggerEvent event) {
       eventRef.set(event);
       try {
         actionStarted.await();
@@ -626,8 +626,9 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
     newNode = cluster.startJettySolrRunner();
     // it should fire again but not complete yet
     await = actionStarted.await(60, TimeUnit.SECONDS);
-    AutoScaling.TriggerEvent replayedEvent = eventRef.get();
-    assertTrue(replayedEvent instanceof TriggerEventQueue.QueuedEvent);
+    TriggerEvent replayedEvent = eventRef.get();
+    assertTrue(replayedEvent.getProperty(TriggerEventQueue.ENQUEUE_TIME) != null);
+    assertTrue(replayedEvent.getProperty(TriggerEventQueue.DEQUEUE_TIME) != null);
     await = actionCompleted.await(10, TimeUnit.SECONDS);
     assertTrue(triggerFired.get());
   }