You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2021/03/10 09:58:06 UTC

[lucene] 01/06: SOLR-14749: ClusterSingleton part of the PR.

This is an automated email from the ASF dual-hosted git repository.

dweiss pushed a commit to branch jira/solr-14749-cluster-singleton
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit b4fa024f684b05786352a2ca0407e19ea8ac1810
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Oct 8 13:37:16 2020 +0200

    SOLR-14749: ClusterSingleton part of the PR.
---
 .../apache/solr/api/CustomContainerPlugins.java    |  21 +-
 .../org/apache/solr/cloud/ClusterSingleton.java    |  31 ++-
 .../src/java/org/apache/solr/cloud/Overseer.java   |   7 -
 .../apache/solr/cluster/events/ClusterEvent.java   |  57 -----
 .../solr/cluster/events/ClusterEventListener.java  |  36 ---
 .../solr/cluster/events/ClusterEventProducer.java  | 100 --------
 .../events/ClusterPropertiesChangedEvent.java      |  39 ---
 .../solr/cluster/events/CollectionsAddedEvent.java |  39 ---
 .../cluster/events/CollectionsRemovedEvent.java    |  39 ---
 .../apache/solr/cluster/events/NodesDownEvent.java |  39 ---
 .../apache/solr/cluster/events/NodesUpEvent.java   |  39 ---
 .../solr/cluster/events/ReplicasDownEvent.java     |  41 ----
 .../events/impl/ClusterEventProducerImpl.java      | 241 ------------------
 .../impl/CollectionsRepairEventListener.java       | 185 --------------
 .../solr/cluster/events/impl/package-info.java     |  23 --
 .../apache/solr/cluster/events/package-info.java   |  23 --
 .../java/org/apache/solr/core/CoreContainer.java   |  60 -----
 .../test/org/apache/solr/cloud/OverseerTest.java   |   3 -
 .../solr/cluster/events/AllEventsListener.java     |  76 ------
 .../cluster/events/ClusterEventProducerTest.java   | 273 ---------------------
 .../impl/CollectionsRepairEventListenerTest.java   | 132 ----------
 .../apache/solr/handler/TestContainerPlugin.java   |  69 ++++++
 22 files changed, 94 insertions(+), 1479 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java b/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
index ec3e4e3..119f651 100644
--- a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
+++ b/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
@@ -34,8 +34,6 @@ import org.apache.lucene.util.ResourceLoaderAware;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.request.beans.PluginMeta;
 import org.apache.solr.cloud.ClusterSingleton;
-import org.apache.solr.cluster.events.ClusterEvent;
-import org.apache.solr.cluster.events.ClusterEventListener;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.annotation.JsonProperty;
@@ -181,9 +179,10 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
       if (instance instanceof ClusterSingleton) {
         ClusterSingleton singleton = (ClusterSingleton) instance;
         coreContainer.getClusterSingletons().getSingletons().put(singleton.getName(), singleton);
-        // easy check to see if we should immediately start this singleton
-        if (coreContainer.getClusterEventProducer() != null &&
-            coreContainer.getClusterEventProducer().isRunning()) {
+        // check to see if we should immediately start this singleton
+        if (coreContainer.getZkController() != null &&
+            coreContainer.getZkController().getOverseer() != null &&
+            !coreContainer.getZkController().getOverseer().isClosed()) {
           try {
             singleton.start();
           } catch (Exception exc) {
@@ -191,15 +190,6 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
           }
         }
       }
-      if (instance instanceof ClusterEventListener) {
-        // XXX nocommit obtain a list of supported event types from the config
-        ClusterEvent.EventType[] types = ClusterEvent.EventType.values();
-        try {
-          coreContainer.getClusterEventProducer().registerListener((ClusterEventListener) instance, types);
-        } catch (Exception exc) {
-          log.warn("Exception adding ClusterEventListener {}: {}", newApiInfo, exc);
-        }
-      }
     }
     if (oldApiInfo != null) {
       // stop & unregister the old api
@@ -209,9 +199,6 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
         singleton.stop();
         coreContainer.getClusterSingletons().getSingletons().remove(singleton.getName());
       }
-      if (instance instanceof ClusterEventListener) {
-        coreContainer.getClusterEventProducer().unregisterListener((ClusterEventListener) instance);
-      }
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/ClusterSingleton.java b/solr/core/src/java/org/apache/solr/cloud/ClusterSingleton.java
index 1ae1eed..70b02fb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ClusterSingleton.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ClusterSingleton.java
@@ -17,12 +17,12 @@
 package org.apache.solr.cloud;
 
 /**
- * Intended for {@link org.apache.solr.core.CoreContainer} plugins that should be
- * enabled only one instance per cluster.
- * <p>Components that implement this interface are always in one of two states:
+ * Intended for plugins that should be enabled only one instance per cluster.
+ * <p>Components that implement this interface are always in one of these states:
  * <ul>
  *   <li>STOPPED - the default state. The component is idle and does not perform
  *   any functions. It should also avoid holding any resources.</li>
+ *   <li>STARTING - </li>
  *   <li>RUNNING - the component is active.</li>
  * </ul>
  * <p>Components must be prepared to change these states multiple times in their
@@ -33,28 +33,39 @@ package org.apache.solr.cloud;
  */
 public interface ClusterSingleton {
 
+  enum State {
+    /** Component is idle. */
+    STOPPED,
+    /** Component is starting. */
+    STARTING,
+    /** Component is active. */
+    RUNNING,
+    /** Component is stopping. */
+    STOPPING
+  }
+
   /**
    * Unique name of this singleton. Used for registration.
    */
   String getName();
 
   /**
-   * Start the operation of the component. On return the component is assumed
-   * to be in the RUNNING state.
+   * Start the operation of the component. Initially this method should set
+   * the state to STARTING, and on success it should set the state to RUNNING.
    * @throws Exception on startup errors. The component should revert to the
    * STOPPED state.
    */
   void start() throws Exception;
 
   /**
-   * Returns true if the component is in the RUNNING state, false otherwise.
+   * Returns the current state of the component.
    */
-  boolean isRunning();
+  State getState();
 
   /**
-   * Stop the operation of the component. On return the component is assumed
-   * to be in the STOPPED state. Components should also avoid holding any resources
-   * in the STOPPED state.
+   * Stop the operation of the component. Initially this method should set
+   * the state to STOPPING, and on return it should set the state to STOPPED.
+   * Components should also avoid holding any resource when in STOPPED state.
    */
   void stop();
 }
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index 2465f8a..0650902 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -49,7 +49,6 @@ import org.apache.solr.cloud.overseer.ReplicaMutator;
 import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.cloud.overseer.ZkStateWriter;
 import org.apache.solr.cloud.overseer.ZkWriteCommand;
-import org.apache.solr.cluster.events.ClusterEventListener;
 import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.SolrCloseable;
 import org.apache.solr.common.SolrException;
@@ -801,9 +800,6 @@ public class Overseer implements SolrCloseable {
       singletons.getSingletons().forEach((name, singleton) -> {
         try {
           singleton.start();
-          if (singleton instanceof ClusterEventListener) {
-            getCoreContainer().getClusterEventProducer().registerListener((ClusterEventListener) singleton);
-          }
         } catch (Exception e) {
           log.warn("Exception starting ClusterSingleton {}: {}", singleton, e);
         }
@@ -826,9 +822,6 @@ public class Overseer implements SolrCloseable {
       return;
     }
     singletons.getSingletons().forEach((name, singleton) -> {
-      if (singleton instanceof ClusterEventListener) {
-        getCoreContainer().getClusterEventProducer().unregisterListener((ClusterEventListener) singleton);
-      }
       singleton.stop();
     });
   }
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEvent.java b/solr/core/src/java/org/apache/solr/cluster/events/ClusterEvent.java
deleted file mode 100644
index 2dc7a32..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEvent.java
+++ /dev/null
@@ -1,57 +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.cluster.events;
-
-import org.apache.solr.common.MapWriter;
-
-import java.io.IOException;
-import java.time.Instant;
-
-/**
- * Cluster-level event.
- */
-public interface ClusterEvent extends MapWriter {
-
-  enum EventType {
-    /** One or more nodes went down. */
-    NODES_DOWN,
-    /** One or more nodes went up. */
-    NODES_UP,
-    /** One or more collections have been added. */
-    COLLECTIONS_ADDED,
-    /** One or more collections have been removed. */
-    COLLECTIONS_REMOVED,
-    /** One or more replicas went down. */
-    REPLICAS_DOWN,
-    /** Cluster properties have changed. */
-    CLUSTER_PROPERTIES_CHANGED
-    // other types? eg. Overseer leader change, shard leader change,
-    // node overload (eg. CPU / MEM circuit breakers tripped)?
-  }
-
-  /** Get event type. */
-  EventType getType();
-
-  /** Get event timestamp. This is the instant when the event was generated (not necessarily when
-   * the underlying condition first occurred). */
-  Instant getTimestamp();
-
-  default void writeMap(EntryWriter ew) throws IOException {
-    ew.put("type", getType());
-    ew.put("timestamp", getTimestamp().toEpochMilli());
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventListener.java b/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventListener.java
deleted file mode 100644
index 592f118..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventListener.java
+++ /dev/null
@@ -1,36 +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.cluster.events;
-
-import org.apache.solr.cloud.ClusterSingleton;
-
-/**
- * Components that want to be notified of cluster-wide events should use this.
- *
- * XXX should this work only for ClusterSingleton-s? some types of events may be
- * XXX difficult (or pointless) to propagate to every node.
- */
-public interface ClusterEventListener extends ClusterSingleton {
-
-  /**
-   * Handle the event. Implementations should be non-blocking - if any long
-   * processing is needed it should be performed asynchronously.
-   * @param event cluster event
-   */
-  void onEvent(ClusterEvent event);
-
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java b/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
deleted file mode 100644
index 1c2327b..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
+++ /dev/null
@@ -1,100 +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.cluster.events;
-
-import org.apache.solr.cloud.ClusterSingleton;
-
-import java.util.Collections;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * Component that produces {@link ClusterEvent} instances.
- */
-public interface ClusterEventProducer extends ClusterSingleton {
-
-  String PLUGIN_NAME = "clusterEventProducer";
-
-  default String getName() {
-    return PLUGIN_NAME;
-  }
-
-  /**
-   * Returns a modifiable map of event types and listeners to process events
-   * of a given type.
-   */
-  Map<ClusterEvent.EventType, Set<ClusterEventListener>> getEventListeners();
-
-  /**
-   * Register an event listener for processing the specified event types.
-   * @param listener non-null listener. If the same instance of the listener is
-   *                 already registered it will be ignored.
-   * @param eventTypes non-empty array of event types that this listener
-   *                   is being registered for. If this is null or empty then all types will be used.
-   */
-  default void registerListener(ClusterEventListener listener, ClusterEvent.EventType... eventTypes) throws Exception {
-    Objects.requireNonNull(listener);
-    if (eventTypes == null || eventTypes.length == 0) {
-      eventTypes = ClusterEvent.EventType.values();
-    }
-    for (ClusterEvent.EventType type : eventTypes) {
-      Set<ClusterEventListener> perType = getEventListeners().computeIfAbsent(type, t -> ConcurrentHashMap.newKeySet());
-      perType.add(listener);
-    }
-  }
-
-  /**
-   * Unregister an event listener.
-   * @param listener non-null listener.
-   */
-  default void unregisterListener(ClusterEventListener listener) {
-    Objects.requireNonNull(listener);
-    getEventListeners().forEach((type, listeners) -> {
-      listeners.remove(listener);
-    });
-  }
-
-  /**
-   * Unregister an event listener for specified event types.
-   * @param listener non-null listener.
-   * @param eventTypes event types from which the listener will be unregistered. If this
-   *                   is null or empty then all event types will be used
-   */
-  default void unregisterListener(ClusterEventListener listener, ClusterEvent.EventType... eventTypes) {
-    Objects.requireNonNull(listener);
-    if (eventTypes == null || eventTypes.length == 0) {
-      eventTypes = ClusterEvent.EventType.values();
-    }
-    for (ClusterEvent.EventType type : eventTypes) {
-      getEventListeners()
-          .getOrDefault(type, Collections.emptySet())
-          .remove(listener);
-    }
-  }
-
-  /**
-   * Fire an event. This method will call registered listeners that subscribed to the
-   * type of event being passed.
-   * @param event cluster event
-   */
-  default void fireEvent(ClusterEvent event) {
-    getEventListeners().getOrDefault(event.getType(), Collections.emptySet())
-        .forEach(listener -> listener.onEvent(event));
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/ClusterPropertiesChangedEvent.java b/solr/core/src/java/org/apache/solr/cluster/events/ClusterPropertiesChangedEvent.java
deleted file mode 100644
index ad9c0b8..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/ClusterPropertiesChangedEvent.java
+++ /dev/null
@@ -1,39 +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.cluster.events;
-
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * Event generated when {@link org.apache.solr.common.cloud.ZkStateReader#CLUSTER_PROPS} is modified.
- */
-public interface ClusterPropertiesChangedEvent extends ClusterEvent {
-
-  @Override
-  default EventType getType() {
-    return EventType.CLUSTER_PROPERTIES_CHANGED;
-  }
-
-  Map<String, Object> getNewClusterProperties();
-
-  @Override
-  default void writeMap(EntryWriter ew) throws IOException {
-    ClusterEvent.super.writeMap(ew);
-    ew.put("newClusterProperties", getNewClusterProperties());
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/CollectionsAddedEvent.java b/solr/core/src/java/org/apache/solr/cluster/events/CollectionsAddedEvent.java
deleted file mode 100644
index 78046f8..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/CollectionsAddedEvent.java
+++ /dev/null
@@ -1,39 +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.cluster.events;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-/**
- * Event generated when some collections have been added.
- */
-public interface CollectionsAddedEvent extends ClusterEvent {
-
-  @Override
-  default EventType getType() {
-    return EventType.COLLECTIONS_ADDED;
-  }
-
-  Iterator<String> getCollectionNames();
-
-  @Override
-  default void writeMap(EntryWriter ew) throws IOException {
-    ClusterEvent.super.writeMap(ew);
-    ew.put("collectionNames", getCollectionNames());
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/CollectionsRemovedEvent.java b/solr/core/src/java/org/apache/solr/cluster/events/CollectionsRemovedEvent.java
deleted file mode 100644
index a93be4c..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/CollectionsRemovedEvent.java
+++ /dev/null
@@ -1,39 +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.cluster.events;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-/**
- * Event generated when some collections have been removed.
- */
-public interface CollectionsRemovedEvent extends ClusterEvent {
-
-  @Override
-  default EventType getType() {
-    return EventType.COLLECTIONS_REMOVED;
-  }
-
-  Iterator<String> getCollectionNames();
-
-  @Override
-  default void writeMap(EntryWriter ew) throws IOException {
-    ClusterEvent.super.writeMap(ew);
-    ew.put("collectionNames", getCollectionNames());
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/NodesDownEvent.java b/solr/core/src/java/org/apache/solr/cluster/events/NodesDownEvent.java
deleted file mode 100644
index 5001ccb..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/NodesDownEvent.java
+++ /dev/null
@@ -1,39 +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.cluster.events;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-/**
- * Event generated when some nodes went down.
- */
-public interface NodesDownEvent extends ClusterEvent {
-
-  @Override
-  default EventType getType() {
-    return EventType.NODES_DOWN;
-  }
-
-  Iterator<String> getNodeNames();
-
-  @Override
-  default void writeMap(EntryWriter ew) throws IOException {
-    ClusterEvent.super.writeMap(ew);
-    ew.put("nodeNames", getNodeNames());
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/NodesUpEvent.java b/solr/core/src/java/org/apache/solr/cluster/events/NodesUpEvent.java
deleted file mode 100644
index fa08f85..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/NodesUpEvent.java
+++ /dev/null
@@ -1,39 +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.cluster.events;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-/**
- * Event generated when some nodes went up.
- */
-public interface NodesUpEvent extends ClusterEvent {
-
-  @Override
-  default EventType getType() {
-    return EventType.NODES_UP;
-  }
-
-  Iterator<String> getNodeNames();
-
-  @Override
-  default void writeMap(EntryWriter ew) throws IOException {
-    ClusterEvent.super.writeMap(ew);
-    ew.put("nodeNames", getNodeNames());
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/ReplicasDownEvent.java b/solr/core/src/java/org/apache/solr/cluster/events/ReplicasDownEvent.java
deleted file mode 100644
index 1d3ce9b..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/ReplicasDownEvent.java
+++ /dev/null
@@ -1,41 +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.cluster.events;
-
-import org.apache.solr.common.cloud.Replica;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-/**
- * Event generated when some replicas went down.
- */
-public interface ReplicasDownEvent extends ClusterEvent {
-
-  @Override
-  default EventType getType() {
-    return EventType.REPLICAS_DOWN;
-  }
-
-  Iterator<Replica> getReplicas();
-
-  @Override
-  default void writeMap(EntryWriter ew) throws IOException {
-    ClusterEvent.super.writeMap(ew);
-    ew.put("replicas", getReplicas());
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerImpl.java b/solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerImpl.java
deleted file mode 100644
index 034fa8a..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerImpl.java
+++ /dev/null
@@ -1,241 +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.cluster.events.impl;
-
-import java.lang.invoke.MethodHandles;
-import java.time.Instant;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.solr.cloud.ZkController;
-import org.apache.solr.cluster.events.ClusterPropertiesChangedEvent;
-import org.apache.solr.cluster.events.ClusterEvent;
-import org.apache.solr.cluster.events.ClusterEventListener;
-import org.apache.solr.cluster.events.ClusterEventProducer;
-import org.apache.solr.cloud.ClusterSingleton;
-import org.apache.solr.cluster.events.CollectionsAddedEvent;
-import org.apache.solr.cluster.events.CollectionsRemovedEvent;
-import org.apache.solr.cluster.events.NodesDownEvent;
-import org.apache.solr.cluster.events.NodesUpEvent;
-import org.apache.solr.common.cloud.CloudCollectionsListener;
-import org.apache.solr.common.cloud.ClusterPropertiesListener;
-import org.apache.solr.common.cloud.LiveNodesListener;
-import org.apache.solr.core.CoreContainer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Implementation of {@link ClusterEventProducer}.
- * <h3>Implementation notes</h3>
- * <p>For each cluster event relevant listeners are always invoked sequentially
- * (not in parallel) and in arbitrary order. This means that if any listener blocks the
- * processing other listeners may be invoked much later or not at all.</p>
- */
-public class ClusterEventProducerImpl implements ClusterEventProducer, ClusterSingleton {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  private final Map<ClusterEvent.EventType, Set<ClusterEventListener>> listeners = new HashMap<>();
-  private CoreContainer coreContainer;
-  private LiveNodesListener liveNodesListener;
-  private CloudCollectionsListener cloudCollectionsListener;
-  private ClusterPropertiesListener clusterPropertiesListener;
-  private ZkController zkController;
-  private volatile boolean running;
-
-  private final Set<ClusterEvent.EventType> supportedEvents =
-      new HashSet<>(Arrays.asList(
-          ClusterEvent.EventType.NODES_DOWN,
-          ClusterEvent.EventType.NODES_UP,
-          ClusterEvent.EventType.COLLECTIONS_ADDED,
-          ClusterEvent.EventType.COLLECTIONS_REMOVED,
-          ClusterEvent.EventType.CLUSTER_PROPERTIES_CHANGED
-      ));
-
-  public ClusterEventProducerImpl(CoreContainer coreContainer) {
-    this.coreContainer = coreContainer;
-  }
-
-  // ClusterSingleton lifecycle methods
-  @Override
-  public void start() {
-    if (coreContainer == null) {
-      liveNodesListener = null;
-      cloudCollectionsListener = null;
-      clusterPropertiesListener = null;
-      return;
-    }
-    this.zkController = this.coreContainer.getZkController();
-
-    // clean up any previous instances
-    doStop();
-
-    // register liveNodesListener
-    liveNodesListener = (oldNodes, newNodes) -> {
-      // already closed but still registered
-      if (!running) {
-        // remove the listener
-        return true;
-      }
-      // spurious event, ignore but keep listening
-      if (oldNodes.equals(newNodes)) {
-        return false;
-      }
-      final Instant now = Instant.now();
-      final Set<String> downNodes = new HashSet<>(oldNodes);
-      downNodes.removeAll(newNodes);
-      if (!downNodes.isEmpty()) {
-        fireEvent(new NodesDownEvent() {
-          @Override
-          public Iterator<String> getNodeNames() {
-            return downNodes.iterator();
-          }
-
-          @Override
-          public Instant getTimestamp() {
-            return now;
-          }
-        });
-      }
-      final Set<String> upNodes = new HashSet<>(newNodes);
-      upNodes.removeAll(oldNodes);
-      if (!upNodes.isEmpty()) {
-        fireEvent(new NodesUpEvent() {
-          @Override
-          public Iterator<String> getNodeNames() {
-            return upNodes.iterator();
-          }
-
-          @Override
-          public Instant getTimestamp() {
-            return now;
-          }
-        });
-      }
-      return false;
-    };
-    zkController.zkStateReader.registerLiveNodesListener(liveNodesListener);
-
-    cloudCollectionsListener = ((oldCollections, newCollections) -> {
-      if (oldCollections.equals(newCollections)) {
-        return;
-      }
-      final Instant now = Instant.now();
-      final Set<String> removed = new HashSet<>(oldCollections);
-      removed.removeAll(newCollections);
-      if (!removed.isEmpty()) {
-        fireEvent(new CollectionsRemovedEvent() {
-          @Override
-          public Iterator<String> getCollectionNames() {
-            return removed.iterator();
-          }
-
-          @Override
-          public Instant getTimestamp() {
-            return now;
-          }
-        });
-      }
-      final Set<String> added = new HashSet<>(newCollections);
-      added.removeAll(oldCollections);
-      if (!added.isEmpty()) {
-        fireEvent(new CollectionsAddedEvent() {
-          @Override
-          public Iterator<String> getCollectionNames() {
-            return added.iterator();
-          }
-
-          @Override
-          public Instant getTimestamp() {
-            return now;
-          }
-        });
-      }
-    });
-    zkController.zkStateReader.registerCloudCollectionsListener(cloudCollectionsListener);
-
-    clusterPropertiesListener = (newProperties) -> {
-      fireEvent(new ClusterPropertiesChangedEvent() {
-        final Instant now = Instant.now();
-        @Override
-        public Map<String, Object> getNewClusterProperties() {
-          return newProperties;
-        }
-
-        @Override
-        public Instant getTimestamp() {
-          return now;
-        }
-      });
-      return false;
-    };
-    zkController.zkStateReader.registerClusterPropertiesListener(clusterPropertiesListener);
-
-    // XXX register collection state listener?
-    // XXX not sure how to efficiently monitor for REPLICA_DOWN events
-
-    running = true;
-  }
-
-  @Override
-  public boolean isRunning() {
-    return running;
-  }
-
-  @Override
-  public void stop() {
-    doStop();
-    running = false;
-  }
-
-  private void doStop() {
-    if (liveNodesListener != null) {
-      zkController.zkStateReader.removeLiveNodesListener(liveNodesListener);
-    }
-    if (cloudCollectionsListener != null) {
-      zkController.zkStateReader.removeCloudCollectionsListener(cloudCollectionsListener);
-    }
-    if (clusterPropertiesListener != null) {
-      zkController.zkStateReader.removeClusterPropertiesListener(clusterPropertiesListener);
-    }
-    liveNodesListener = null;
-    cloudCollectionsListener = null;
-    clusterPropertiesListener = null;
-  }
-
-  @Override
-  public void registerListener(ClusterEventListener listener, ClusterEvent.EventType... eventTypes) throws Exception {
-    try {
-      for (ClusterEvent.EventType type : eventTypes) {
-        if (!supportedEvents.contains(type)) {
-          log.warn("event type {} not supported yet.", type);
-        }
-      }
-    } catch (Throwable e) {
-      throw new Exception(e);
-    }
-    ClusterEventProducer.super.registerListener(listener, eventTypes);
-  }
-
-  @Override
-  public Map<ClusterEvent.EventType, Set<ClusterEventListener>> getEventListeners() {
-    return listeners;
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java b/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java
deleted file mode 100644
index 42dcde3..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java
+++ /dev/null
@@ -1,185 +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.cluster.events.impl;
-
-import java.io.IOException;
-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.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.cloud.api.collections.Assign;
-import org.apache.solr.cluster.events.ClusterEvent;
-import org.apache.solr.cluster.events.ClusterEventListener;
-import org.apache.solr.cluster.events.NodesDownEvent;
-import org.apache.solr.cluster.events.ReplicasDownEvent;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.ReplicaPosition;
-import org.apache.solr.core.CoreContainer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This is an illustration how to re-implement the combination of 8x
- * NodeLostTrigger and AutoAddReplicasPlanAction to maintain the collection's replication factor.
- * <p>NOTE: there's no support for 'waitFor' yet.</p>
- * <p>NOTE 2: this functionality would be probably more reliable when executed also as a
- * periodically scheduled check - both as a reactive (listener) and proactive (scheduled) measure.</p>
- */
-public class CollectionsRepairEventListener implements ClusterEventListener {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  public static final String PLUGIN_NAME = "collectionsRepairListener";
-  private static final String ASYNC_ID_PREFIX = "_async_" + PLUGIN_NAME;
-  private static final AtomicInteger counter = new AtomicInteger();
-
-  private final SolrClient solrClient;
-  private final SolrCloudManager solrCloudManager;
-
-  private volatile boolean running = false;
-
-  public CollectionsRepairEventListener(CoreContainer cc) {
-    this.solrClient = cc.getSolrClientCache().getCloudSolrClient(cc.getZkController().getZkClient().getZkServerAddress());
-    this.solrCloudManager = cc.getZkController().getSolrCloudManager();
-  }
-
-  @Override
-  public String getName() {
-    return PLUGIN_NAME;
-  }
-
-  @Override
-  public void onEvent(ClusterEvent event) {
-    if (!isRunning()) {
-      // ignore the event
-      return;
-    }
-    switch (event.getType()) {
-      case NODES_DOWN:
-        handleNodesDown((NodesDownEvent) event);
-        break;
-      case REPLICAS_DOWN:
-        handleReplicasDown((ReplicasDownEvent) event);
-        break;
-      default:
-        log.warn("Unsupported event {}, ignoring...", event);
-    }
-  }
-
-  private void handleNodesDown(NodesDownEvent event) {
-    // collect all lost replicas
-    // collection / positions
-    Map<String, List<ReplicaPosition>> newPositions = new HashMap<>();
-    try {
-      ClusterState clusterState = solrCloudManager.getClusterStateProvider().getClusterState();
-      Set<String> lostNodeNames = new HashSet<>();
-      event.getNodeNames().forEachRemaining(lostNodeNames::add);
-      clusterState.forEachCollection(coll -> {
-        // shard / type / count
-        Map<String, Map<Replica.Type, AtomicInteger>> lostReplicas = new HashMap<>();
-        coll.forEachReplica((shard, replica) -> {
-          if (lostNodeNames.contains(replica.getNodeName())) {
-            lostReplicas.computeIfAbsent(shard, s -> new HashMap<>())
-                .computeIfAbsent(replica.type, t -> new AtomicInteger())
-                .incrementAndGet();
-          }
-        });
-        Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(solrCloudManager, clusterState, coll);
-        lostReplicas.forEach((shard, types) -> {
-          Assign.AssignRequestBuilder assignRequestBuilder = new Assign.AssignRequestBuilder()
-              .forCollection(coll.getName())
-              .forShard(Collections.singletonList(shard));
-          types.forEach((type, count) -> {
-            switch (type) {
-              case NRT:
-                assignRequestBuilder.assignNrtReplicas(count.get());
-                break;
-              case PULL:
-                assignRequestBuilder.assignPullReplicas(count.get());
-                break;
-              case TLOG:
-                assignRequestBuilder.assignTlogReplicas(count.get());
-                break;
-            }
-          });
-          Assign.AssignRequest assignRequest = assignRequestBuilder.build();
-          try {
-            List<ReplicaPosition> positions = assignStrategy.assign(solrCloudManager, assignRequest);
-            newPositions.put(coll.getName(), positions);
-          } catch (Exception e) {
-            log.warn("Exception computing positions for {}/{}: {}", coll.getName(), shard, e);
-            return;
-          }
-        });
-      });
-    } catch (IOException e) {
-      log.warn("Exception getting cluster state", e);
-      return;
-    }
-
-    // send ADDREPLICA admin requests for each lost replica
-    // XXX should we use 'async' for that, to avoid blocking here?
-    List<CollectionAdminRequest.AddReplica> addReplicas = new ArrayList<>();
-    newPositions.forEach((collection, positions) -> {
-      positions.forEach(position -> {
-        CollectionAdminRequest.AddReplica addReplica = CollectionAdminRequest
-            .addReplicaToShard(collection, position.shard, position.type);
-        addReplica.setNode(position.node);
-        addReplica.setAsyncId(ASYNC_ID_PREFIX + counter.incrementAndGet());
-        addReplicas.add(addReplica);
-      });
-    });
-    addReplicas.forEach(addReplica -> {
-      try {
-        solrClient.request(addReplica);
-      } catch (Exception e) {
-        log.warn("Exception calling ADDREPLICA {}: {}", addReplica.getParams().toQueryString(), e);
-      }
-    });
-
-    // ... and DELETERPLICA for lost ones?
-  }
-
-  private void handleReplicasDown(ReplicasDownEvent event) {
-    // compute new placements for all replicas that went down
-    // send ADDREPLICA admin request for each lost replica
-  }
-
-  @Override
-  public void start() throws Exception {
-    running = true;
-  }
-
-  @Override
-  public boolean isRunning() {
-    return running;
-  }
-
-  @Override
-  public void stop() {
-    running = false;
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/impl/package-info.java b/solr/core/src/java/org/apache/solr/cluster/events/impl/package-info.java
deleted file mode 100644
index 2c115b6..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/impl/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
- 
-/** 
- * Default implementation of {@link org.apache.solr.cluster.events.ClusterEventProducer}.
- */
-package org.apache.solr.cluster.events.impl;
-
-
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/package-info.java b/solr/core/src/java/org/apache/solr/cluster/events/package-info.java
deleted file mode 100644
index a334a00..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/events/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
- 
-/** 
- * Interfaces defining support for cluster-level event generation and processing.
- */
-package org.apache.solr.cluster.events;
-
-
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 1427dad..ac664e0 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -74,10 +74,6 @@ import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ClusterSingleton;
 import org.apache.solr.cloud.OverseerTaskQueue;
 import org.apache.solr.cloud.ZkController;
-import org.apache.solr.cluster.events.ClusterEvent;
-import org.apache.solr.cluster.events.ClusterEventListener;
-import org.apache.solr.cluster.events.ClusterEventProducer;
-import org.apache.solr.cluster.events.impl.ClusterEventProducerImpl;
 import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -203,45 +199,6 @@ public class CoreContainer {
     }
   }
 
-  /**
-   * This class helps in handling the initial registration of plugin-based listeners,
-   * when both the final {@link ClusterEventProducer} implementation and listeners
-   * are configured using plugins.
-   */
-  public static class InitialClusterEventProducer implements ClusterEventProducer {
-    Map<ClusterEvent.EventType, Set<ClusterEventListener>> initialListeners = new HashMap<>();
-
-    @Override
-    public Map<ClusterEvent.EventType, Set<ClusterEventListener>> getEventListeners() {
-      return initialListeners;
-    }
-
-    public void transferListeners(ClusterEventProducer target) {
-      initialListeners.forEach((type, listeners) -> {
-        listeners.forEach(listener -> {
-          try {
-            target.registerListener(listener, type);
-          } catch (Exception e) {
-            log.warn("Unable to register event listener for type {}: {}", type, e);
-          }
-        });
-      });
-    }
-
-    @Override
-    public void start() throws Exception {
-    }
-
-    @Override
-    public boolean isRunning() {
-      return false;
-    }
-
-    @Override
-    public void stop() {
-    }
-  }
-
   private volatile PluginBag<SolrRequestHandler> containerHandlers = new PluginBag<>(SolrRequestHandler.class, null);
 
   /**
@@ -317,8 +274,6 @@ public class CoreContainer {
 
   private volatile SolrClientCache solrClientCache;
 
-  private volatile ClusterEventProducer clusterEventProducer = new InitialClusterEventProducer();
-
   private final ObjectCache objectCache = new ObjectCache();
 
   private final ClusterSingletons clusterSingletons = new ClusterSingletons();
@@ -979,17 +934,6 @@ public class CoreContainer {
           clusterSingletons.singletonMap.put(handlerName, (ClusterSingleton) handler);
         }
       });
-      // create the ClusterEventProducer
-      InitialClusterEventProducer initialClusterEventProducer = (InitialClusterEventProducer) clusterEventProducer;
-      CustomContainerPlugins.ApiInfo clusterEventProducerInfo = customContainerPlugins.getPlugin(ClusterEventProducer.PLUGIN_NAME);
-      if (clusterEventProducerInfo != null) {
-        clusterEventProducer = (ClusterEventProducer) clusterEventProducerInfo.getInstance();
-      } else {
-        clusterEventProducer = new ClusterEventProducerImpl(this);
-        clusterSingletons.singletonMap.put(ClusterEventProducer.PLUGIN_NAME, clusterEventProducer);
-      }
-      // transfer those listeners that were already registered to the initial impl
-      initialClusterEventProducer.transferListeners(clusterEventProducer);
 
       clusterSingletons.setReady();
       zkSys.getZkController().checkOverseerDesignate();
@@ -2195,10 +2139,6 @@ public class CoreContainer {
     return clusterSingletons;
   }
 
-  public ClusterEventProducer getClusterEventProducer() {
-    return clusterEventProducer;
-  }
-
   static {
     ExecutorUtil.addThreadLocalProvider(SolrRequestInfo.getInheritableThreadLocalProvider());
   }
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
index f6c07c1..125a4cd 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
@@ -50,7 +50,6 @@ import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
 import org.apache.solr.cloud.overseer.NodeMutator;
 import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.cloud.overseer.ZkWriteCommand;
-import org.apache.solr.cluster.events.impl.ClusterEventProducerImpl;
 import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -1427,8 +1426,6 @@ public class OverseerTest extends SolrTestCaseJ4 {
     // don't wait for all singletons
     singletons.setReady();
     FieldSetter.setField(mockAlwaysUpCoreContainer, CoreContainer.class.getDeclaredField("clusterSingletons"), singletons);
-    ClusterEventProducerImpl clusterEventProducer = new ClusterEventProducerImpl(mockAlwaysUpCoreContainer);
-    when(mockAlwaysUpCoreContainer.getClusterEventProducer()).thenReturn(clusterEventProducer);
     FieldSetter.setField(zkController, ZkController.class.getDeclaredField("zkClient"), zkClient);
     FieldSetter.setField(zkController, ZkController.class.getDeclaredField("cc"), mockAlwaysUpCoreContainer);
     when(zkController.getCoreContainer()).thenReturn(mockAlwaysUpCoreContainer);
diff --git a/solr/core/src/test/org/apache/solr/cluster/events/AllEventsListener.java b/solr/core/src/test/org/apache/solr/cluster/events/AllEventsListener.java
deleted file mode 100644
index 8da6bea..0000000
--- a/solr/core/src/test/org/apache/solr/cluster/events/AllEventsListener.java
+++ /dev/null
@@ -1,76 +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.cluster.events;
-
-import org.junit.Assert;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-/**
- *
- */
-public class AllEventsListener implements ClusterEventListener {
-  CountDownLatch eventLatch = new CountDownLatch(1);
-  ClusterEvent.EventType expectedType;
-  Map<ClusterEvent.EventType, List<ClusterEvent>> events = new HashMap<>();
-
-  @Override
-  public String getName() {
-    return "allEventsListener";
-  }
-
-  @Override
-  public void onEvent(ClusterEvent event) {
-    events.computeIfAbsent(event.getType(), type -> new ArrayList<>()).add(event);
-    if (event.getType() == expectedType) {
-      eventLatch.countDown();
-    }
-  }
-
-  public void setExpectedType(ClusterEvent.EventType expectedType) {
-    this.expectedType = expectedType;
-    eventLatch = new CountDownLatch(1);
-  }
-
-  public void waitForExpectedEvent(int timeoutSeconds) throws InterruptedException {
-    boolean await = eventLatch.await(timeoutSeconds, TimeUnit.SECONDS);
-    if (!await) {
-      Assert.fail("Timed out waiting for expected event " + expectedType);
-    }
-  }
-
-  @Override
-  public void start() throws Exception {
-
-  }
-
-  @Override
-  public boolean isRunning() {
-    return false;
-  }
-
-  @Override
-  public void stop() {
-
-  }
-}
diff --git a/solr/core/src/test/org/apache/solr/cluster/events/ClusterEventProducerTest.java b/solr/core/src/test/org/apache/solr/cluster/events/ClusterEventProducerTest.java
deleted file mode 100644
index ea6c5f5..0000000
--- a/solr/core/src/test/org/apache/solr/cluster/events/ClusterEventProducerTest.java
+++ /dev/null
@@ -1,273 +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.cluster.events;
-
-import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.V2Request;
-import org.apache.solr.client.solrj.request.beans.PluginMeta;
-import org.apache.solr.client.solrj.response.V2Response;
-import org.apache.solr.cloud.SolrCloudTestCase;
-import org.apache.solr.common.cloud.ClusterProperties;
-import org.apache.solr.common.util.Utils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.invoke.MethodHandles;
-import java.time.Instant;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import static java.util.Collections.singletonMap;
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-
-/**
- *
- */
-public class ClusterEventProducerTest extends SolrCloudTestCase {
-
-  private AllEventsListener eventsListener;
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    configureCluster(3)
-        .addConfig("conf", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
-        .configure();
-  }
-
-  @Before
-  public void setUp() throws Exception  {
-    System.setProperty("enable.packages", "true");
-    super.setUp();
-    cluster.deleteAllCollections();
-    eventsListener = new AllEventsListener();
-    cluster.getOpenOverseer().getCoreContainer().getClusterEventProducer().registerListener(eventsListener);
-  }
-
-  @After
-  public void teardown() {
-    System.clearProperty("enable.packages");
-    if (eventsListener != null) {
-      cluster.getOpenOverseer().getCoreContainer().getClusterEventProducer().unregisterListener(eventsListener);
-    }
-  }
-
-  @Test
-  public void testEvents() throws Exception {
-
-    // NODES_DOWN
-
-    eventsListener.setExpectedType(ClusterEvent.EventType.NODES_DOWN);
-
-    // don't kill Overseer
-    JettySolrRunner nonOverseerJetty = null;
-    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
-      if (cluster.getOpenOverseer().getCoreContainer().getZkController().getNodeName().equals(jetty.getNodeName())) {
-        continue;
-      }
-      nonOverseerJetty = jetty;
-      break;
-    }
-    String nodeName = nonOverseerJetty.getNodeName();
-    cluster.stopJettySolrRunner(nonOverseerJetty);
-    cluster.waitForJettyToStop(nonOverseerJetty);
-    eventsListener.waitForExpectedEvent(10);
-    assertNotNull("should be NODES_DOWN events", eventsListener.events.get(ClusterEvent.EventType.NODES_DOWN));
-    List<ClusterEvent> events = eventsListener.events.get(ClusterEvent.EventType.NODES_DOWN);
-    assertEquals("should be one NODES_DOWN event", 1, events.size());
-    ClusterEvent event = events.get(0);
-    assertEquals("should be NODES_DOWN event type", ClusterEvent.EventType.NODES_DOWN, event.getType());
-    NodesDownEvent nodesDown = (NodesDownEvent) event;
-    assertEquals("should be node " + nodeName, nodeName, nodesDown.getNodeNames().next());
-
-    // NODES_UP
-    eventsListener.setExpectedType(ClusterEvent.EventType.NODES_UP);
-    JettySolrRunner newNode = cluster.startJettySolrRunner();
-    cluster.waitForNode(newNode, 60);
-    eventsListener.waitForExpectedEvent(10);
-    assertNotNull("should be NODES_UP events", eventsListener.events.get(ClusterEvent.EventType.NODES_UP));
-    events = eventsListener.events.get(ClusterEvent.EventType.NODES_UP);
-    assertEquals("should be one NODES_UP event", 1, events.size());
-    event = events.get(0);
-    assertEquals("should be NODES_UP event type", ClusterEvent.EventType.NODES_UP, event.getType());
-    NodesUpEvent nodesUp = (NodesUpEvent) event;
-    assertEquals("should be node " + newNode.getNodeName(), newNode.getNodeName(), nodesUp.getNodeNames().next());
-
-    // COLLECTIONS_ADDED
-    eventsListener.setExpectedType(ClusterEvent.EventType.COLLECTIONS_ADDED);
-    String collection = "testNodesEvent_collection";
-    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collection, "conf", 1, 1);
-    cluster.getSolrClient().request(create);
-    cluster.waitForActiveCollection(collection, 1, 1);
-    eventsListener.waitForExpectedEvent(10);
-    assertNotNull("should be COLLECTIONS_ADDED events", eventsListener.events.get(ClusterEvent.EventType.COLLECTIONS_ADDED));
-    events = eventsListener.events.get(ClusterEvent.EventType.COLLECTIONS_ADDED);
-    assertEquals("should be one COLLECTIONS_ADDED event", 1, events.size());
-    event = events.get(0);
-    assertEquals("should be COLLECTIONS_ADDED event type", ClusterEvent.EventType.COLLECTIONS_ADDED, event.getType());
-    CollectionsAddedEvent collectionsAdded = (CollectionsAddedEvent) event;
-    assertEquals("should be collection " + collection, collection, collectionsAdded.getCollectionNames().next());
-
-    // COLLECTIONS_REMOVED
-    eventsListener.setExpectedType(ClusterEvent.EventType.COLLECTIONS_REMOVED);
-    CollectionAdminRequest.Delete delete = CollectionAdminRequest.deleteCollection(collection);
-    cluster.getSolrClient().request(delete);
-    eventsListener.waitForExpectedEvent(10);
-    assertNotNull("should be COLLECTIONS_REMOVED events", eventsListener.events.get(ClusterEvent.EventType.COLLECTIONS_REMOVED));
-    events = eventsListener.events.get(ClusterEvent.EventType.COLLECTIONS_REMOVED);
-    assertEquals("should be one COLLECTIONS_REMOVED event", 1, events.size());
-    event = events.get(0);
-    assertEquals("should be COLLECTIONS_REMOVED event type", ClusterEvent.EventType.COLLECTIONS_REMOVED, event.getType());
-    CollectionsRemovedEvent collectionsRemoved = (CollectionsRemovedEvent) event;
-    assertEquals("should be collection " + collection, collection, collectionsRemoved.getCollectionNames().next());
-
-    // CLUSTER_CONFIG_CHANGED
-    eventsListener.setExpectedType(ClusterEvent.EventType.CLUSTER_PROPERTIES_CHANGED);
-    ClusterProperties clusterProperties = new ClusterProperties(cluster.getZkClient());
-    Map<String, Object> oldProps = new HashMap<>(clusterProperties.getClusterProperties());
-    clusterProperties.setClusterProperty("ext.foo", "bar");
-    eventsListener.waitForExpectedEvent(10);
-    assertNotNull("should be CLUSTER_CONFIG_CHANGED events", eventsListener.events.get(ClusterEvent.EventType.CLUSTER_PROPERTIES_CHANGED));
-    events = eventsListener.events.get(ClusterEvent.EventType.CLUSTER_PROPERTIES_CHANGED);
-    assertEquals("should be one CLUSTER_CONFIG_CHANGED event", 1, events.size());
-    event = events.get(0);
-    assertEquals("should be CLUSTER_CONFIG_CHANGED event type", ClusterEvent.EventType.CLUSTER_PROPERTIES_CHANGED, event.getType());
-    ClusterPropertiesChangedEvent propertiesChanged = (ClusterPropertiesChangedEvent) event;
-    Map<String, Object> newProps = propertiesChanged.getNewClusterProperties();
-    assertEquals("new properties wrong value of the 'ext.foo' property: " + newProps,
-        "bar", newProps.get("ext.foo"));
-
-    // unset the property
-    eventsListener.setExpectedType(ClusterEvent.EventType.CLUSTER_PROPERTIES_CHANGED);
-    clusterProperties.setClusterProperty("ext.foo", null);
-    eventsListener.waitForExpectedEvent(10);
-    assertNotNull("should be CLUSTER_CONFIG_CHANGED events", eventsListener.events.get(ClusterEvent.EventType.CLUSTER_PROPERTIES_CHANGED));
-    events = eventsListener.events.get(ClusterEvent.EventType.CLUSTER_PROPERTIES_CHANGED);
-    assertEquals("should be two CLUSTER_CONFIG_CHANGED events", 2, events.size());
-    event = events.get(1);
-    assertEquals("should be CLUSTER_CONFIG_CHANGED event type", ClusterEvent.EventType.CLUSTER_PROPERTIES_CHANGED, event.getType());
-    propertiesChanged = (ClusterPropertiesChangedEvent) event;
-    assertEquals("new properties should not have 'ext.foo' property: " + propertiesChanged.getNewClusterProperties(),
-        null, propertiesChanged.getNewClusterProperties().get("ext.foo"));
-
-  }
-
-  private static CountDownLatch dummyEventLatch = new CountDownLatch(1);
-  private static ClusterEvent lastEvent = null;
-
-  public static class DummyEventListener implements ClusterEventListener {
-    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-    boolean running = false;
-    @Override
-    public void onEvent(ClusterEvent event) {
-      if (!running) {
-        log.debug("skipped event, not running: {}", event);
-        return;
-      }
-      if (event.getType() == ClusterEvent.EventType.COLLECTIONS_ADDED ||
-          event.getType() == ClusterEvent.EventType.COLLECTIONS_REMOVED) {
-        log.debug("recorded event {}", Utils.toJSONString(event));
-        lastEvent = event;
-        dummyEventLatch.countDown();
-      } else {
-        log.debug("skipped event, wrong type: {}", event.getType());
-      }
-    }
-
-    @Override
-    public String getName() {
-      return "dummy";
-    }
-
-    @Override
-    public void start() throws Exception {
-      log.debug("starting {}", Integer.toHexString(hashCode()));
-      running = true;
-    }
-
-    @Override
-    public boolean isRunning() {
-      return running;
-    }
-
-    @Override
-    public void stop() {
-      log.debug("stopping {}", Integer.toHexString(hashCode()));
-      running = false;
-    }
-  }
-
-  @Test
-  public void testListenerPlugins() throws Exception {
-    PluginMeta plugin = new PluginMeta();
-    plugin.name = "testplugin";
-    plugin.klass = DummyEventListener.class.getName();
-    V2Request req = new V2Request.Builder("/cluster/plugin")
-        .forceV2(true)
-        .withMethod(POST)
-        .withPayload(singletonMap("add", plugin))
-        .build();
-    V2Response rsp = req.process(cluster.getSolrClient());
-    //just check if the plugin is indeed registered
-    V2Request readPluginState = new V2Request.Builder("/cluster/plugin")
-        .forceV2(true)
-        .withMethod(GET)
-        .build();
-    rsp = readPluginState.process(cluster.getSolrClient());
-    assertEquals(DummyEventListener.class.getName(), rsp._getStr("/plugin/testplugin/class", null));
-
-    String collection = "testListenerPlugins_collection";
-    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collection, "conf", 1, 1);
-    cluster.getSolrClient().request(create);
-    cluster.waitForActiveCollection(collection, 1, 1);
-    boolean await = dummyEventLatch.await(30, TimeUnit.SECONDS);
-    if (!await) {
-      fail("Timed out waiting for COLLECTIONS_ADDED event, " + collection);
-    }
-    assertNotNull("lastEvent should be COLLECTIONS_ADDED", lastEvent);
-    assertEquals("lastEvent should be COLLECTIONS_ADDED", ClusterEvent.EventType.COLLECTIONS_ADDED, lastEvent.getType());
-    // verify timestamp
-    Instant now = Instant.now();
-    assertTrue("timestamp of the event is in the future", now.isAfter(lastEvent.getTimestamp()));
-    assertEquals(collection, ((CollectionsAddedEvent)lastEvent).getCollectionNames().next());
-
-    dummyEventLatch = new CountDownLatch(1);
-    lastEvent = null;
-
-    CollectionAdminRequest.Delete delete = CollectionAdminRequest.deleteCollection(collection);
-    cluster.getSolrClient().request(delete);
-    await = dummyEventLatch.await(30, TimeUnit.SECONDS);
-    if (!await) {
-      fail("Timed out waiting for COLLECTIONS_REMOVED event, " + collection);
-    }
-    assertNotNull("lastEvent should be COLLECTIONS_REMOVED", lastEvent);
-    assertEquals("lastEvent should be COLLECTIONS_REMOVED", ClusterEvent.EventType.COLLECTIONS_REMOVED, lastEvent.getType());
-    // verify timestamp
-    now = Instant.now();
-    assertTrue("timestamp of the event is in the future", now.isAfter(lastEvent.getTimestamp()));
-    assertEquals(collection, ((CollectionsRemovedEvent)lastEvent).getCollectionNames().next());
-  }
-}
diff --git a/solr/core/src/test/org/apache/solr/cluster/events/impl/CollectionsRepairEventListenerTest.java b/solr/core/src/test/org/apache/solr/cluster/events/impl/CollectionsRepairEventListenerTest.java
deleted file mode 100644
index a764051..0000000
--- a/solr/core/src/test/org/apache/solr/cluster/events/impl/CollectionsRepairEventListenerTest.java
+++ /dev/null
@@ -1,132 +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.cluster.events.impl;
-
-import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.cloud.SolrCloudTestCase;
-import org.apache.solr.cluster.events.AllEventsListener;
-import org.apache.solr.cluster.events.ClusterEvent;
-import org.apache.solr.cluster.events.ClusterEventListener;
-import org.apache.solr.core.CoreContainer;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-/**
- *
- */
-public class CollectionsRepairEventListenerTest extends SolrCloudTestCase {
-
-  public static class CollectionsRepairWrapperListener implements ClusterEventListener {
-    final CollectionsRepairEventListener delegate;
-
-    CountDownLatch completed = new CountDownLatch(1);
-
-    CollectionsRepairWrapperListener(CoreContainer cc) throws Exception {
-      delegate = new CollectionsRepairEventListener(cc);
-    }
-
-    @Override
-    public void onEvent(ClusterEvent event) {
-      delegate.onEvent(event);
-      completed.countDown();
-    }
-
-    @Override
-    public String getName() {
-      return "wrapperListener";
-    }
-
-    @Override
-    public void start() throws Exception {
-      delegate.start();
-    }
-
-    @Override
-    public boolean isRunning() {
-      return delegate.isRunning();
-    }
-
-    @Override
-    public void stop() {
-      delegate.stop();
-    }
-  }
-
-  private static AllEventsListener eventsListener = new AllEventsListener();
-  private static CollectionsRepairWrapperListener repairListener;
-
-  private static int NUM_NODES = 3;
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    configureCluster(NUM_NODES)
-        .addConfig("conf", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
-        .configure();
-    CoreContainer cc = cluster.getOpenOverseer().getCoreContainer();
-    cc.getClusterEventProducer()
-        .registerListener(eventsListener, ClusterEvent.EventType.values());
-    repairListener = new CollectionsRepairWrapperListener(cc);
-    cc.getClusterEventProducer()
-        .registerListener(repairListener, ClusterEvent.EventType.NODES_DOWN);
-    repairListener.start();
-  }
-
-  @Before
-  public void setUp() throws Exception  {
-    super.setUp();
-    cluster.deleteAllCollections();
-  }
-
-  @Test
-  public void testCollectionRepair() throws Exception {
-    eventsListener.setExpectedType(ClusterEvent.EventType.COLLECTIONS_ADDED);
-    String collection = "testCollectionRepair_collection";
-    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collection, "conf", 1, 3);
-    cluster.getSolrClient().request(create);
-    cluster.waitForActiveCollection(collection, 1, 3);
-    eventsListener.waitForExpectedEvent(10);
-    eventsListener.setExpectedType(ClusterEvent.EventType.NODES_DOWN);
-
-    // don't kill Overseer
-    JettySolrRunner nonOverseerJetty = null;
-    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
-      if (cluster.getOpenOverseer().getCoreContainer().getZkController().getNodeName().equals(jetty.getNodeName())) {
-        continue;
-      }
-      nonOverseerJetty = jetty;
-      break;
-    }
-    String nodeName = nonOverseerJetty.getNodeName();
-    cluster.stopJettySolrRunner(nonOverseerJetty);
-    cluster.waitForJettyToStop(nonOverseerJetty);
-    eventsListener.waitForExpectedEvent(10);
-    cluster.waitForActiveCollection(collection, 1, 2);
-
-    // wait for completed processing in the repair listener
-    boolean await = repairListener.completed.await(60, TimeUnit.SECONDS);
-    if (!await) {
-      fail("Timeout waiting for the processing to complete");
-    }
-    cluster.waitForActiveCollection(collection, 1, 3);
-  }
-}
diff --git a/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java b/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
index bd9bf7c..6f7a18a 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
@@ -37,10 +37,12 @@ import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.client.solrj.request.beans.Package;
 import org.apache.solr.client.solrj.request.beans.PluginMeta;
 import org.apache.solr.client.solrj.response.V2Response;
+import org.apache.solr.cloud.ClusterSingleton;
 import org.apache.solr.cloud.MiniSolrCloudCluster;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.NavigableObject;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.filestore.PackageStoreAPI;
 import org.apache.solr.filestore.TestDistribPackageStore;
@@ -92,6 +94,9 @@ public class TestContainerPlugin extends SolrCloudTestCase {
       expectError(req, cluster.getSolrClient(), errPath, "No method with @Command in class");
 
       //test with an invalid class
+      // XXX (ab) in order to support ClusterSingleton we allow adding
+      // plugins without Api EndPoints
+
 //      plugin.klass = C1.class.getName();
 //      expectError(req, cluster.getSolrClient(), errPath, "No @EndPoints");
 
@@ -170,6 +175,31 @@ public class TestContainerPlugin extends SolrCloudTestCase {
           .withMethod(GET)
           .build()
           .process(cluster.getSolrClient()));
+
+      // test ClusterSingleton plugin
+      plugin.name = "clusterSingleton";
+      plugin.klass = C6.class.getName();
+      req.process(cluster.getSolrClient());
+
+      //just check if the plugin is indeed registered
+      readPluginState = new V2Request.Builder("/cluster/plugin")
+          .forceV2(true)
+          .withMethod(GET)
+          .build();
+      rsp = readPluginState.process(cluster.getSolrClient());
+      assertEquals(C6.class.getName(), rsp._getStr("/plugin/clusterSingleton/class", null));
+
+      assertTrue("ccProvided", C6.ccProvided);
+      assertTrue("startCalled", C6.startCalled);
+      assertFalse("stopCalled", C6.stopCalled);
+      // kill the Overseer leader
+      for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+        if (!jetty.getCoreContainer().getZkController().getOverseer().isClosed()) {
+          cluster.stopJettySolrRunner(jetty);
+          cluster.waitForJettyToStop(jetty);
+        }
+      }
+      assertTrue("stopCalled", C6.stopCalled);
     } finally {
       cluster.shutdown();
     }
@@ -289,6 +319,45 @@ public class TestContainerPlugin extends SolrCloudTestCase {
     }
   }
 
+  public static class C6 implements ClusterSingleton {
+    static boolean startCalled = false;
+    static boolean stopCalled = false;
+    static boolean ccProvided = false;
+
+    private State state = State.STOPPED;
+
+    public C6(CoreContainer cc) {
+      if (cc != null) {
+        ccProvided = true;
+      }
+    }
+
+    @Override
+    public String getName() {
+      return "C6";
+    }
+
+    @Override
+    public void start() throws Exception {
+      state = State.STARTING;
+      startCalled = true;
+      state = State.RUNNING;
+    }
+
+    @Override
+    public State getState() {
+      return state;
+    }
+
+    @Override
+    public void stop() {
+      state = State.STOPPING;
+      stopCalled = true;
+      state = State.STOPPED;
+    }
+  }
+
+
   public static class C5 implements ResourceLoaderAware {
     static ByteBuffer classData;
     private  SolrResourceLoader resourceLoader;