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:05 UTC

[lucene] branch jira/solr-14749-cluster-singleton created (now a356db9)

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

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


      at a356db9  SOLR-14749: Change back this var to the original name.

This branch includes the following new commits:

     new b4fa024  SOLR-14749: ClusterSingleton part of the PR.
     new ef13cd9  SOLR-14749: Use allowEmpty instead of required.
     new 56105d0  move out direct dependency on CLusterSingleton from CustomContainerPlugins
     new 2e93f57  Merge branch 'master' into jira/solr-14749-cluster-singleton
     new e549845  SOLR-14749: Move much of the singleton mgmgt into the utility class.
     new a356db9  SOLR-14749: Change back this var to the original name.

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[lucene] 03/06: move out direct dependency on CLusterSingleton from CustomContainerPlugins

Posted by dw...@apache.org.
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 56105d0beb0eaa8a54370494f7c6dab92818fe6f
Author: noblepaul <no...@gmail.com>
AuthorDate: Thu Oct 15 13:28:28 2020 +1100

    move out direct dependency on CLusterSingleton from CustomContainerPlugins
---
 .../apache/solr/api/CustomContainerPlugins.java    | 29 ++++++++++++++++++++++
 1 file changed, 29 insertions(+)

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 119f651..616fb9d 100644
--- a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
+++ b/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
+import java.util.concurrent.CopyOnWriteArrayList;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.lucene.util.ResourceLoaderAware;
@@ -59,6 +60,8 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
   private final ObjectMapper mapper = SolrJacksonAnnotationInspector.createObjectMapper();
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  private final List<PluginRegistryListener> listeners = new CopyOnWriteArrayList<>();
+
   final CoreContainer coreContainer;
   final ApiBag containerApiBag;
 
@@ -70,6 +73,13 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
     return false;
   }
 
+  public void registerListener(PluginRegistryListener listener) {
+    listeners.add(listener);
+  }
+  public void unregisterListener(PluginRegistryListener listener) {
+    listeners.remove(listener);
+  }
+
   public CustomContainerPlugins(CoreContainer coreContainer, ApiBag apiBag) {
     this.coreContainer = coreContainer;
     this.containerApiBag = apiBag;
@@ -113,6 +123,7 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
       if (e.getValue() == Diff.REMOVED) {
         ApiInfo apiInfo = currentPlugins.remove(e.getKey());
         if (apiInfo == null) continue;
+        listeners.forEach(listener -> listener.deleted(apiInfo));
         handleClusterSingleton(null, apiInfo);
         for (ApiHolder holder : apiInfo.holders) {
           Api old = containerApiBag.unregister(holder.api.getEndPoint().method()[0],
@@ -143,6 +154,8 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
             containerApiBag.register(holder, getTemplateVars(apiInfo.info));
           }
           currentPlugins.put(e.getKey(), apiInfo);
+          final ApiInfo apiInfoFinal = apiInfo;
+          listeners.forEach(listener -> listener.added(apiInfoFinal));
           handleClusterSingleton(apiInfo, null);
         } else {
           //this plugin is being updated
@@ -151,6 +164,8 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
             //register all new paths
             containerApiBag.register(holder, getTemplateVars(apiInfo.info));
           }
+          final ApiInfo apiInfoFinal = apiInfo;
+          listeners.forEach(listener -> listener.modified(old, apiInfoFinal));
           handleClusterSingleton(apiInfo, old);
           if (old != null) {
             //this is an update of the plugin. But, it is possible that
@@ -240,6 +255,7 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
   @SuppressWarnings({"rawtypes"})
   public class ApiInfo implements ReflectMapWriter {
     List<ApiHolder> holders;
+
     @JsonProperty
     private final PluginMeta info;
 
@@ -265,6 +281,9 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
       return instance;
     }
 
+    public PluginMeta getInfo() {
+      return info.copy();
+    }
     @SuppressWarnings({"unchecked","rawtypes"})
     public ApiInfo(PluginMeta info, List<String> errs) {
       this.info = info;
@@ -401,4 +420,14 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
 
     return null;
   }
+
+  interface PluginRegistryListener {
+
+    void added(ApiInfo plugin);
+
+    void deleted(ApiInfo plugin);
+
+    void modified(ApiInfo old, ApiInfo replacement);
+
+  }
 }


[lucene] 02/06: SOLR-14749: Use allowEmpty instead of required.

Posted by dw...@apache.org.
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 ef13cd92d42357a3192d33ad9949f46769264a7f
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Mon Oct 12 16:00:51 2020 +0200

    SOLR-14749: Use allowEmpty instead of required.
---
 solr/core/src/java/org/apache/solr/api/AnnotatedApi.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
index 1558f1c..f9f97a4 100644
--- a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
+++ b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
@@ -92,11 +92,11 @@ public class AnnotatedApi extends Api implements PermissionNameProvider , Closea
    * Get a list of Api-s supported by this class.
    * @param theClass class
    * @param obj object of this class (may be null)
-   * @param required if true then an exception is thrown if no Api-s can be retrieved, if false
+   * @param allowEmpty if false then an exception is thrown if no Api-s can be retrieved, if true
    *                then absence of Api-s is silently ignored.
    * @return list of discovered Api-s
    */
-  public static List<Api> getApis(Class<? extends Object> theClass , Object obj, boolean required)  {
+  public static List<Api> getApis(Class<? extends Object> theClass , Object obj, boolean allowEmpty)  {
     Class<?> klas = null;
     try {
       klas = MethodHandles.publicLookup().accessClass(theClass);
@@ -131,7 +131,7 @@ public class AnnotatedApi extends Api implements PermissionNameProvider , Closea
         SpecProvider specProvider = readSpec(endPoint, Collections.singletonList(m));
         apis.add(new AnnotatedApi(specProvider, endPoint, Collections.singletonMap("", cmd), null));
       }
-      if (required && apis.isEmpty()) {
+      if (!allowEmpty && apis.isEmpty()) {
         throw new RuntimeException("Invalid Class : " + klas.getName() + " No @EndPoints");
       }
 


[lucene] 06/06: SOLR-14749: Change back this var to the original name.

Posted by dw...@apache.org.
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 a356db9133d04117ca8f21ee22457f4736b1cdb3
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Mon Oct 19 14:10:50 2020 +0200

    SOLR-14749: Change back this var to the original name.
---
 .../org/apache/solr/handler/admin/ContainerPluginsApi.java     | 10 +++++-----
 .../java/org/apache/solr/packagemanager/PackageManager.java    |  2 +-
 2 files changed, 6 insertions(+), 6 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java b/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java
index ad95423..0c7a487 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java
@@ -51,7 +51,7 @@ import static org.apache.lucene.util.IOUtils.closeWhileHandlingException;
 public class ContainerPluginsApi {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static final String PLUGINS = "plugin";
+  public static final String PLUGIN = "plugin";
   private final Supplier<SolrZkClient> zkClientSupplier;
   private final CoreContainer coreContainer;
   public final Read readAPI = new Read();
@@ -67,7 +67,7 @@ public class ContainerPluginsApi {
         path = "/cluster/plugin",
         permission = PermissionNameProvider.Name.COLL_READ_PERM)
     public void list(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
-      rsp.add(PLUGINS, plugins(zkClientSupplier));
+      rsp.add(PLUGIN, plugins(zkClientSupplier));
     }
   }
 
@@ -151,7 +151,7 @@ public class ContainerPluginsApi {
     SolrZkClient zkClient = zkClientSupplier.get();
     try {
       Map<String, Object> clusterPropsJson = (Map<String, Object>) Utils.fromJSON(zkClient.getData(ZkStateReader.CLUSTER_PROPS, null, new Stat(), true));
-      return (Map<String, Object>) clusterPropsJson.computeIfAbsent(PLUGINS, o -> new LinkedHashMap<>());
+      return (Map<String, Object>) clusterPropsJson.computeIfAbsent(PLUGIN, o -> new LinkedHashMap<>());
     } catch (KeeperException.NoNodeException e) {
       return new LinkedHashMap<>();
     } catch (KeeperException | InterruptedException e) {
@@ -165,9 +165,9 @@ public class ContainerPluginsApi {
       zkClientSupplier.get().atomicUpdate(ZkStateReader.CLUSTER_PROPS, bytes -> {
         Map rawJson = bytes == null ? new LinkedHashMap() :
             (Map) Utils.fromJSON(bytes);
-        Map pluginsModified = modifier.apply((Map) rawJson.computeIfAbsent(PLUGINS, o -> new LinkedHashMap<>()));
+        Map pluginsModified = modifier.apply((Map) rawJson.computeIfAbsent(PLUGIN, o -> new LinkedHashMap<>()));
         if (pluginsModified == null) return null;
-        rawJson.put(PLUGINS, pluginsModified);
+        rawJson.put(PLUGIN, pluginsModified);
         return Utils.toJSON(rawJson);
       });
     } catch (KeeperException | InterruptedException e) {
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java b/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java
index 044cf8c..6f2f618 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java
@@ -232,7 +232,7 @@ public class PackageManager implements Closeable {
       }
     }
     @SuppressWarnings({"unchecked"})
-    Map<String, Object> clusterPlugins = (Map<String, Object>) result.getOrDefault(ContainerPluginsApi.PLUGINS, Collections.emptyMap());
+    Map<String, Object> clusterPlugins = (Map<String, Object>) result.getOrDefault(ContainerPluginsApi.PLUGIN, Collections.emptyMap());
     for (String key : clusterPlugins.keySet()) {
       // Map<String, String> pluginMeta = (Map<String, String>) clusterPlugins.get(key);
       PluginMeta pluginMeta;


[lucene] 05/06: SOLR-14749: Move much of the singleton mgmgt into the utility class.

Posted by dw...@apache.org.
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 e549845ee72b7d547713904d3217141ffacee22f
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Mon Oct 19 13:40:53 2020 +0200

    SOLR-14749: Move much of the singleton mgmgt into the utility class.
---
 .../apache/solr/api/CustomContainerPlugins.java    |  49 ++-----
 .../src/java/org/apache/solr/cloud/Overseer.java   |  46 +-----
 .../org/apache/solr/core/ClusterSingletons.java    | 162 +++++++++++++++++++++
 .../java/org/apache/solr/core/CoreContainer.java   |  41 ++----
 .../test/org/apache/solr/cloud/OverseerTest.java   |   3 +-
 5 files changed, 186 insertions(+), 115 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 616fb9d..5f4d9a7 100644
--- a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
+++ b/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
@@ -34,7 +34,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 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.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.annotation.JsonProperty;
@@ -57,9 +56,10 @@ import static org.apache.lucene.util.IOUtils.closeWhileHandlingException;
 import static org.apache.solr.common.util.Utils.makeMap;
 
 public class CustomContainerPlugins implements ClusterPropertiesListener, MapWriter {
-  private final ObjectMapper mapper = SolrJacksonAnnotationInspector.createObjectMapper();
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  private final ObjectMapper mapper = SolrJacksonAnnotationInspector.createObjectMapper();
+
   private final List<PluginRegistryListener> listeners = new CopyOnWriteArrayList<>();
 
   final CoreContainer coreContainer;
@@ -124,7 +124,6 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
         ApiInfo apiInfo = currentPlugins.remove(e.getKey());
         if (apiInfo == null) continue;
         listeners.forEach(listener -> listener.deleted(apiInfo));
-        handleClusterSingleton(null, apiInfo);
         for (ApiHolder holder : apiInfo.holders) {
           Api old = containerApiBag.unregister(holder.api.getEndPoint().method()[0],
               getActualPath(apiInfo, holder.api.getEndPoint().path()[0]));
@@ -156,7 +155,6 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
           currentPlugins.put(e.getKey(), apiInfo);
           final ApiInfo apiInfoFinal = apiInfo;
           listeners.forEach(listener -> listener.added(apiInfoFinal));
-          handleClusterSingleton(apiInfo, null);
         } else {
           //this plugin is being updated
           ApiInfo old = currentPlugins.put(e.getKey(), apiInfo);
@@ -166,7 +164,6 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
           }
           final ApiInfo apiInfoFinal = apiInfo;
           listeners.forEach(listener -> listener.modified(old, apiInfoFinal));
-          handleClusterSingleton(apiInfo, old);
           if (old != null) {
             //this is an update of the plugin. But, it is possible that
             // some paths are remved in the newer version of the plugin
@@ -187,36 +184,6 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
     }
   }
 
-  private void handleClusterSingleton(ApiInfo newApiInfo, ApiInfo oldApiInfo) {
-    if (newApiInfo != null) {
-      // register new api
-      Object instance = newApiInfo.getInstance();
-      if (instance instanceof ClusterSingleton) {
-        ClusterSingleton singleton = (ClusterSingleton) instance;
-        coreContainer.getClusterSingletons().getSingletons().put(singleton.getName(), singleton);
-        // 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) {
-            log.warn("Exception starting ClusterSingleton {}: {}", newApiInfo, exc);
-          }
-        }
-      }
-    }
-    if (oldApiInfo != null) {
-      // stop & unregister the old api
-      Object instance = oldApiInfo.getInstance();
-      if (instance instanceof ClusterSingleton) {
-        ClusterSingleton singleton = (ClusterSingleton) instance;
-        singleton.stop();
-        coreContainer.getClusterSingletons().getSingletons().remove(singleton.getName());
-      }
-    }
-  }
-
   private static String getActualPath(ApiInfo apiInfo, String path) {
     path = path.replaceAll("\\$path-prefix", apiInfo.info.pathPrefix);
     path = path.replaceAll("\\$plugin-name", apiInfo.info.name);
@@ -329,7 +296,7 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
       }
 
       try {
-        List<Api> apis = AnnotatedApi.getApis(klas, null, false);
+        List<Api> apis = AnnotatedApi.getApis(klas, null, true);
         for (Object api : apis) {
           EndPoint endPoint = ((AnnotatedApi) api).getEndPoint();
           if (endPoint.path().length > 1 || endPoint.method().length > 1) {
@@ -381,7 +348,7 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
         }
       }
       this.holders = new ArrayList<>();
-      for (Api api : AnnotatedApi.getApis(instance.getClass(), instance, false)) {
+      for (Api api : AnnotatedApi.getApis(instance.getClass(), instance, true)) {
         holders.add(new ApiHolder((AnnotatedApi) api));
       }
     }
@@ -421,12 +388,18 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
     return null;
   }
 
-  interface PluginRegistryListener {
+  /**
+   * Listener for notifications about added / deleted / modified plugins.
+   */
+  public interface PluginRegistryListener {
 
+    /** Called when a new plugin is added. */
     void added(ApiInfo plugin);
 
+    /** Called when an existing plugin is deleted. */
     void deleted(ApiInfo plugin);
 
+    /** Called when an existing plugin is replaced. */
     void modified(ApiInfo old, ApiInfo replacement);
 
   }
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 4c206ae..fc0d0eb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -29,8 +29,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.function.BiConsumer;
 
 import org.apache.lucene.util.Version;
@@ -657,7 +655,7 @@ public class Overseer implements SolrCloseable {
       }
     });
 
-    startClusterSingletons();
+    getCoreContainer().getClusterSingletons().startClusterSingletons();
 
     assert ObjectReleaseTracker.track(this);
   }
@@ -781,50 +779,10 @@ public class Overseer implements SolrCloseable {
   /**
    * Start {@link ClusterSingleton} plugins when we become the leader.
    */
-  public void startClusterSingletons() {
-    CoreContainer.ClusterSingletons singletons = getCoreContainer().getClusterSingletons();
-    final Runnable initializer = () -> {
-      if (isClosed()) {
-        return;
-      }
-      try {
-        singletons.waitUntilReady(60, TimeUnit.SECONDS);
-      } catch (InterruptedException e) {
-        log.warn("Interrupted initialization of ClusterSingleton-s");
-        return;
-      } catch (TimeoutException te) {
-        log.warn("Timed out during initialization of ClusterSingleton-s");
-        return;
-      }
-      singletons.getSingletons().forEach((name, singleton) -> {
-        try {
-          singleton.start();
-        } catch (Exception e) {
-          log.warn("Exception starting ClusterSingleton {}: {}", singleton, e);
-        }
-      });
-    };
-    if (singletons.isReady()) {
-      // wait until all singleton-s are ready for the first startup
-      getCoreContainer().runAsync(initializer);
-    } else {
-      initializer.run();
-    }
-  }
 
   /**
    * Stop {@link ClusterSingleton} plugins when we lose leadership.
    */
-  private void stopClusterSingletons() {
-    CoreContainer.ClusterSingletons singletons = getCoreContainer().getClusterSingletons();
-    if (singletons == null) {
-      return;
-    }
-    singletons.getSingletons().forEach((name, singleton) -> {
-      singleton.stop();
-    });
-  }
-
   public Stats getStats() {
     return stats;
   }
@@ -866,7 +824,7 @@ public class Overseer implements SolrCloseable {
     }
     // stop singletons only on the leader
     if (!this.closed) {
-      stopClusterSingletons();
+      getCoreContainer().getClusterSingletons().stopClusterSingletons();
     }
     this.closed = true;
     doClose();
diff --git a/solr/core/src/java/org/apache/solr/core/ClusterSingletons.java b/solr/core/src/java/org/apache/solr/core/ClusterSingletons.java
new file mode 100644
index 0000000..c929e83
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/ClusterSingletons.java
@@ -0,0 +1,162 @@
+/*
+ * 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.core;
+
+import org.apache.solr.api.CustomContainerPlugins;
+import org.apache.solr.cloud.ClusterSingleton;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+/**
+ * Helper class to manage the initial registration of {@link ClusterSingleton} plugins and
+ * to track the changes in loaded plugins in {@link org.apache.solr.api.CustomContainerPlugins}.
+ */
+public class ClusterSingletons {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final Map<String, ClusterSingleton> singletonMap = new ConcurrentHashMap<>();
+  private final Supplier<Boolean> runSingletons;
+  private final Consumer<Runnable> asyncRunner;
+  private final CustomContainerPlugins.PluginRegistryListener pluginListener;
+
+  // we use this latch to delay the initial startup of singletons, due to
+  // the leader election occurring in parallel with the rest of the load() method.
+  private final CountDownLatch readyLatch = new CountDownLatch(1);
+
+  /**
+   * Create a helper to manage singletons.
+   * @param runSingletons this function returns true when singletons should be running. It's
+   *                      used when adding or modifying existing plugins.
+   * @param asyncRunner async runner that will be used for starting up each singleton.
+   */
+  public ClusterSingletons(Supplier<Boolean> runSingletons, Consumer<Runnable> asyncRunner) {
+    this.runSingletons = runSingletons;
+    this.asyncRunner = asyncRunner;
+    pluginListener = new CustomContainerPlugins.PluginRegistryListener() {
+      @Override
+      public void added(CustomContainerPlugins.ApiInfo plugin) {
+        // register new api
+        Object instance = plugin.getInstance();
+        if (instance instanceof ClusterSingleton) {
+          ClusterSingleton singleton = (ClusterSingleton) instance;
+          singletonMap.put(singleton.getName(), singleton);
+          // check to see if we should immediately start this singleton
+          if (isReady() && runSingletons.get()) {
+            try {
+              singleton.start();
+            } catch (Exception exc) {
+              log.warn("Exception starting ClusterSingleton {}: {}", plugin, exc);
+            }
+          }
+        }
+      }
+
+      @Override
+      public void deleted(CustomContainerPlugins.ApiInfo plugin) {
+        Object instance = plugin.getInstance();
+        if (instance instanceof ClusterSingleton) {
+          ClusterSingleton singleton = (ClusterSingleton) instance;
+          singleton.stop();
+          singletonMap.remove(singleton.getName());
+        }
+      }
+
+      @Override
+      public void modified(CustomContainerPlugins.ApiInfo old, CustomContainerPlugins.ApiInfo replacement) {
+        added(replacement);
+        deleted(old);
+      }
+    };
+  }
+
+  public CustomContainerPlugins.PluginRegistryListener getPluginRegistryListener() {
+    return pluginListener;
+  }
+
+  public Map<String, ClusterSingleton> getSingletons() {
+    return singletonMap;
+  }
+
+  public boolean isReady() {
+    return readyLatch.getCount() == 0;
+  }
+
+  public void setReady() {
+    readyLatch.countDown();
+  }
+
+  public void waitUntilReady(long timeout, TimeUnit timeUnit)
+      throws InterruptedException, TimeoutException {
+    boolean await = readyLatch.await(timeout, timeUnit);
+    if (!await) {
+      throw new TimeoutException("Timed out waiting for ClusterSingletons to become ready.");
+    }
+  }
+
+  /**
+   * Start singletons when the helper is ready and when it's supposed to start
+   * (as determined by {@link #runSingletons} function).
+   */
+  public void startClusterSingletons() {
+    final Runnable initializer = () -> {
+      if (!runSingletons.get()) {
+        return;
+      }
+      try {
+        waitUntilReady(60, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        log.warn("Interrupted initialization of ClusterSingleton-s");
+        return;
+      } catch (TimeoutException te) {
+        log.warn("Timed out during initialization of ClusterSingleton-s");
+        return;
+      }
+      singletonMap.forEach((name, singleton) -> {
+        if (!runSingletons.get()) {
+          return;
+        }
+        try {
+          singleton.start();
+        } catch (Exception e) {
+          log.warn("Exception starting ClusterSingleton {}: {}", singleton, e);
+        }
+      });
+    };
+    if (isReady()) {
+      // wait until all singleton-s are ready for the first startup
+      asyncRunner.accept(initializer);
+    } else {
+      initializer.run();
+    }
+  }
+
+  public void stopClusterSingletons() {
+    singletonMap.forEach((name, singleton) -> {
+      singleton.stop();
+    });
+  }
+}
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 36d5e0d..bdcd0b4 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -39,12 +39,9 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.function.Supplier;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -172,33 +169,6 @@ public class CoreContainer {
     }
   }
 
-  public static class ClusterSingletons {
-    private Map<String, ClusterSingleton> singletonMap = new ConcurrentHashMap<>();
-    // we use this latch to delay the initial startup of singletons, due to
-    // the leader election occurring in parallel with the rest of the load() method.
-    private CountDownLatch readyLatch = new CountDownLatch(1);
-
-    public Map<String, ClusterSingleton> getSingletons() {
-      return singletonMap;
-    }
-
-    public boolean isReady() {
-      return readyLatch.getCount() > 0;
-    }
-
-    public void setReady() {
-      readyLatch.countDown();
-    }
-
-    public void waitUntilReady(long timeout, TimeUnit timeUnit)
-        throws InterruptedException, TimeoutException {
-      boolean await = readyLatch.await(timeout, timeUnit);
-      if (!await) {
-        throw new TimeoutException("Timed out waiting for ClusterSingletons to become ready.");
-      }
-    }
-  }
-
   private volatile PluginBag<SolrRequestHandler> containerHandlers = new PluginBag<>(SolrRequestHandler.class, null);
 
   /**
@@ -276,7 +246,11 @@ public class CoreContainer {
 
   private final ObjectCache objectCache = new ObjectCache();
 
-  private final ClusterSingletons clusterSingletons = new ClusterSingletons();
+  private final ClusterSingletons clusterSingletons = new ClusterSingletons(
+      () -> getZkController() != null &&
+          getZkController().getOverseer() != null &&
+          !getZkController().getOverseer().isClosed(),
+      (r) -> this.runAsync(r));
   private PackageStoreAPI packageStoreAPI;
   private PackageLoader packageLoader;
 
@@ -695,6 +669,8 @@ public class CoreContainer {
       loader.reloadLuceneSPI();
     }
 
+    customContainerPlugins.registerListener(clusterSingletons.getPluginRegistryListener());
+
     packageStoreAPI = new PackageStoreAPI(this);
     containerHandlers.getApiBag().registerObject(packageStoreAPI.readAPI);
     containerHandlers.getApiBag().registerObject(packageStoreAPI.writeAPI);
@@ -915,7 +891,8 @@ public class CoreContainer {
       containerHandlers.keySet().forEach(handlerName -> {
         SolrRequestHandler handler = containerHandlers.get(handlerName);
         if (handler instanceof ClusterSingleton) {
-          clusterSingletons.singletonMap.put(handlerName, (ClusterSingleton) handler);
+          ClusterSingleton singleton = (ClusterSingleton) handler;
+          clusterSingletons.getSingletons().put(singleton.getName(), singleton);
         }
       });
 
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 125a4cd..89fda3e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
@@ -67,6 +67,7 @@ import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CloudConfig;
+import org.apache.solr.core.ClusterSingletons;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrResourceLoader;
@@ -1422,7 +1423,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
         Mockito.withSettings().defaultAnswer(Mockito.CALLS_REAL_METHODS));
     when(mockAlwaysUpCoreContainer.isShutDown()).thenReturn(testDone);  // Allow retry on session expiry
     when(mockAlwaysUpCoreContainer.getResourceLoader()).thenReturn(new SolrResourceLoader());
-    CoreContainer.ClusterSingletons singletons = new CoreContainer.ClusterSingletons();
+    ClusterSingletons singletons = new ClusterSingletons(() -> true, r -> r.run());
     // don't wait for all singletons
     singletons.setReady();
     FieldSetter.setField(mockAlwaysUpCoreContainer, CoreContainer.class.getDeclaredField("clusterSingletons"), singletons);


[lucene] 04/06: Merge branch 'master' into jira/solr-14749-cluster-singleton

Posted by dw...@apache.org.
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 2e93f57e8014f3132ff80e15bd5f588dea966707
Merge: 56105d0 9ab9d20
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Mon Oct 19 11:55:01 2020 +0200

    Merge branch 'master' into jira/solr-14749-cluster-singleton

 .dir-locals.el                                     |   3 +
 .gitignore                                         |   6 +-
 build.gradle                                       |   4 +-
 dev-tools/doap/lucene.rdf                          |   7 +
 dev-tools/doap/solr.rdf                            |   7 +
 dev-tools/scripts/README.md                        |   2 +-
 dev-tools/scripts/addBackcompatIndexes.py          |   5 +-
 dev-tools/scripts/addVersion.py                    |  23 +-
 dev-tools/scripts/buildAndPushRelease.py           |  25 +-
 dev-tools/scripts/releaseWizard.py                 |   5 +-
 dev-tools/scripts/releaseWizard.yaml               |  15 +-
 dev-tools/scripts/scriptutil.py                    |   6 +-
 dev-tools/scripts/smokeTestRelease.py              |  27 +-
 gradle/ant-compat/artifact-naming.gradle           |   6 -
 gradle/documentation/changes-to-html.gradle        |  31 +-
 .../changes-to-html}/ChangesFancyStyle.css         |   0
 .../changes-to-html}/ChangesFixedWidthStyle.css    |   0
 .../changes-to-html}/ChangesSimpleStyle.css        |   0
 .../documentation/changes-to-html}/changes2html.pl |   0
 gradle/documentation/documentation.gradle          |  89 ++-
 gradle/documentation/markdown.gradle               |  31 +-
 gradle/documentation/render-javadoc.gradle         |   3 +-
 gradle/ide/intellij-idea.gradle                    |  16 -
 gradle/releasing.gradle                            |  55 ++
 .../randomization/policies/solr-tests.policy       |   2 +-
 gradle/testing/randomization/policies/tests.policy |   3 -
 gradle/validation/check-broken-links.gradle        |   6 +-
 gradle/validation/jar-checks.gradle                |   5 +-
 .../owasp-dependency-check/exclusions.xml          |  15 -
 lucene/CHANGES.txt                                 |  40 ++
 lucene/MIGRATE.md                                  |  19 +-
 lucene/analysis/README.txt                         |  18 +-
 .../miscellaneous/DropIfFlaggedFilter.java         |  51 ++
 .../miscellaneous/DropIfFlaggedFilterFactory.java  |  71 ++
 .../miscellaneous/TypeAsSynonymFilter.java         |  37 +-
 .../miscellaneous/TypeAsSynonymFilterFactory.java  |  17 +-
 .../org.apache.lucene.analysis.TokenFilterFactory  |   1 +
 .../miscellaneous/TestDropIfFlaggedFilter.java     |  73 +++
 ...ry.java => TestDropIfFlaggedFilterFactory.java} |  30 +-
 .../miscellaneous/TestTypeAsSynonymFilter.java     | 106 +++
 .../TestTypeAsSynonymFilterFactory.java            |   7 +
 .../ja/JapanesePartOfSpeechStopFilterFactory.java  |  19 +-
 .../TestJapanesePartOfSpeechStopFilterFactory.java |  17 +
 lucene/analysis/phonetic/src/java/overview.html    |   2 +-
 .../lucene/codecs/lucene80/Lucene80Codec.java      |   6 +
 .../lucene/codecs/lucene84/Lucene84Codec.java      |   6 +
 .../lucene/codecs/lucene86/Lucene86Codec.java      |   6 +
 .../lucene/codecs/lucene87/Lucene87Codec.java      |   4 +
 .../org/apache/lucene/codecs/lucene87/package.html |  42 ++
 .../services/org.apache.lucene.codecs.Codec        |   1 +
 .../lucene/index/TestBackwardsCompatibility.java   |   7 +-
 .../org/apache/lucene/index/index.8.6.3-cfs.zip    | Bin 0 -> 16419 bytes
 .../org/apache/lucene/index/index.8.6.3-nocfs.zip  | Bin 0 -> 16428 bytes
 .../test/org/apache/lucene/index/sorted.8.6.3.zip  | Bin 0 -> 141031 bytes
 .../benchmark/byTask/tasks/CreateIndexTask.java    |   4 +-
 .../org/apache/lucene/benchmark/package-info.java  |   2 +-
 lucene/build.gradle                                |   2 +-
 .../lucene/codecs/simpletext/SimpleTextCodec.java  |   7 +
 .../simpletext/SimpleTextFieldInfosFormat.java     |  27 +-
 .../codecs/simpletext/SimpleTextVectorFormat.java  |  51 ++
 .../codecs/simpletext/SimpleTextVectorReader.java  | 304 +++++++++
 .../codecs/simpletext/SimpleTextVectorWriter.java  | 148 +++++
 .../codecs/uniformsplit/TestBlockWriter.java       |   3 +
 .../sharedterms/TestSTBlockReader.java             |   3 +
 .../org/apache/lucene/analysis/package-info.java   |   4 +-
 .../src/java/org/apache/lucene/codecs/Codec.java   |   6 +-
 .../java/org/apache/lucene/codecs/FilterCodec.java |   5 +
 .../org/apache/lucene/codecs/VectorFormat.java     |  76 +++
 .../org/apache/lucene/codecs/VectorReader.java     |  55 ++
 .../org/apache/lucene/codecs/VectorWriter.java     | 283 ++++++++
 .../codecs/lucene50/Lucene50FieldInfosFormat.java  |   4 +-
 .../lucene/codecs/lucene50/package-info.java       |   2 +-
 .../codecs/lucene60/Lucene60FieldInfosFormat.java  |   4 +-
 .../lucene/codecs/lucene60/package-info.java       |   2 +-
 .../lucene/codecs/lucene80/package-info.java       |   2 +-
 .../lucene/codecs/lucene84/package-info.java       |   2 +-
 .../lucene/codecs/lucene86/package-info.java       | 398 +-----------
 .../lucene/codecs/lucene87/package-info.java       | 398 +-----------
 .../Lucene90Codec.java}                            |  38 +-
 .../Lucene90FieldInfosFormat.java}                 |  43 +-
 .../codecs/lucene90/Lucene90VectorFormat.java      |  57 ++
 .../codecs/lucene90/Lucene90VectorReader.java      | 345 ++++++++++
 .../codecs/lucene90/Lucene90VectorWriter.java      | 127 ++++
 .../{lucene87 => lucene90}/package-info.java       |  39 +-
 .../java/org/apache/lucene/document/FieldType.java |  28 +
 .../org/apache/lucene/document/VectorField.java    |  99 +++
 .../java/org/apache/lucene/index/CheckIndex.java   |  85 ++-
 .../java/org/apache/lucene/index/CodecReader.java  |  19 +
 .../lucene/index/ConcurrentMergeScheduler.java     |  45 +-
 .../apache/lucene/index/DocValuesLeafReader.java   |   5 +
 .../lucene/index/DocumentsWriterFlushQueue.java    |   4 +-
 .../java/org/apache/lucene/index/FieldInfo.java    |  57 +-
 .../java/org/apache/lucene/index/FieldInfos.java   |  76 ++-
 .../org/apache/lucene/index/FilterCodecReader.java |   6 +
 .../org/apache/lucene/index/FilterLeafReader.java  |   5 +
 .../java/org/apache/lucene/index/IndexWriter.java  |  14 +-
 .../apache/lucene/index/IndexableFieldType.java    |  10 +
 .../org/apache/lucene/index/IndexingChain.java     |  81 ++-
 .../java/org/apache/lucene/index/LeafReader.java   |   4 +
 .../apache/lucene/index/MergeReaderWrapper.java    |   5 +
 .../java/org/apache/lucene/index/MergeState.java   |  11 +
 .../apache/lucene/index/ParallelLeafReader.java    |   7 +
 .../org/apache/lucene/index/ReadersAndUpdates.java |   3 +-
 .../apache/lucene/index/SegmentCoreReaders.java    |  11 +-
 .../org/apache/lucene/index/SegmentMerger.java     | 143 ++--
 .../org/apache/lucene/index/SegmentReader.java     |   6 +
 .../lucene/index/SlowCodecReaderWrapper.java       |  30 +
 .../java/org/apache/lucene/index/VectorValues.java | 276 ++++++++
 .../apache/lucene/index/VectorValuesWriter.java    | 322 +++++++++
 .../org/apache/lucene/search/spans/SpanWeight.java |  21 +-
 .../src/java/org/apache/lucene/util/IOUtils.java   | 147 -----
 .../src/java/org/apache/lucene/util/Version.java   |   7 +
 .../org/apache/lucene/util/automaton/Automata.java |  27 +-
 lucene/core/src/java/overview.html                 |   6 +-
 .../services/org.apache.lucene.codecs.Codec        |   2 +-
 ...tLucene87StoredFieldsFormatHighCompression.java |   7 +-
 .../apache/lucene/index/TestIndexableField.java    |  10 +
 .../lucene/index/TestPendingSoftDeletes.java       |  12 +-
 .../lucene/index/TestSegmentToThreadMapping.java   |   3 +
 .../org/apache/lucene/index/TestVectorValues.java  | 722 +++++++++++++++++++++
 .../lucene/search/spans/TestSpanExplanations.java  |  38 ++
 .../test/org/apache/lucene/util/TestIOUtils.java   | 385 -----------
 .../lucene/util/automaton/TestAutomaton.java       |  36 +
 .../test/org/apache/lucene/util/fst/TestFSTs.java  |   2 +-
 lucene/demo/src/java/overview.html                 |   2 +-
 .../documentation/build.gradle                     |   7 +-
 .../src/assets}/lucene_green_300.gif               | Bin
 .../src/markdown}/index.template.md                |   0
 .../search/highlight/LimitTokenOffsetFilter.java   |   2 +-
 .../search/highlight/TermVectorLeafReader.java     |   8 +-
 lucene/licenses/junit-4.12.jar.sha1                |   1 -
 lucene/licenses/junit-4.13.1.jar.sha1              |   1 +
 lucene/luke/build.gradle                           |  33 +-
 .../apache/lucene/index/memory/MemoryIndex.java    |   9 +-
 lucene/packaging/build.gradle                      | 168 +++++
 lucene/queryparser/build.gradle                    |   2 -
 lucene/site/xsl/index.xsl                          | 117 ----
 .../search/suggest/document/TestSuggestField.java  |   4 +-
 .../lucene/analysis/BaseTokenStreamTestCase.java   | 122 ++--
 .../lucene/index/BaseIndexFileFormatTestCase.java  |   3 +-
 .../apache/lucene/index/MismatchedLeafReader.java  |   2 +
 .../apache/lucene/index/RandomPostingsTester.java  |   4 +-
 .../java/org/apache/lucene/search/QueryUtils.java  |   6 +
 .../org/apache/lucene/util/LuceneTestCase.java     |  13 -
 .../util/TestRuleSetupAndRestoreClassEnv.java      |   6 +-
 .../src/java/org/apache/lucene/util/TestUtil.java  |   9 +-
 lucene/version.properties                          |  10 -
 settings.gradle                                    |   3 +
 solr/CHANGES.txt                                   |  65 +-
 .../solr/prometheus/exporter/MetricsQuery.java     |   2 +-
 .../exporter/PrometheusExporterSettings.java       |   2 +-
 .../prometheus/PrometheusExporterTestBase.java     |   1 +
 .../org/apache/solr/cloud/CloudDescriptor.java     |   2 +-
 .../src/java/org/apache/solr/cloud/Overseer.java   |   1 -
 .../apache/solr/cloud/api/collections/Assign.java  |  72 +-
 .../OverseerCollectionMessageHandler.java          |   3 -
 .../solr/cloud/api/collections/SplitShardCmd.java  |   9 +-
 .../org/apache/solr/cloud/rule/ImplicitSnitch.java |  65 --
 .../apache/solr/cloud/rule/ReplicaAssigner.java    | 452 -------------
 .../src/java/org/apache/solr/cloud/rule/Rule.java  | 395 -----------
 .../solr/cloud/rule/ServerSnitchContext.java       |  59 --
 .../java/org/apache/solr/core/CoreContainer.java   |  30 +-
 .../java/org/apache/solr/core/CoreDescriptor.java  |   2 +-
 .../java/org/apache/solr/core/MetricsConfig.java   |  71 +-
 .../src/java/org/apache/solr/core/PluginInfo.java  |   2 +-
 .../src/java/org/apache/solr/core/SolrConfig.java  |   2 +-
 .../src/java/org/apache/solr/core/SolrCore.java    |   8 -
 .../java/org/apache/solr/core/SolrXmlConfig.java   |  17 +-
 .../java/org/apache/solr/core/XmlConfigFile.java   |   2 +-
 .../apache/solr/handler/ReplicationHandler.java    |  63 +-
 .../org/apache/solr/handler/SolrConfigHandler.java |  33 +-
 .../solr/handler/admin/CollectionsHandler.java     |  50 --
 .../solr/handler/admin/ConfigSetsHandler.java      |  62 +-
 .../solr/handler/admin/CoreAdminOperation.java     |   2 +-
 .../apache/solr/handler/admin/MetricsHandler.java  |  18 +-
 .../solr/handler/admin/MetricsHistoryHandler.java  |  15 +-
 .../handler/component/QueryElevationComponent.java |   4 +-
 .../solr/index/SlowCompositeReaderWrapper.java     |   6 +
 .../org/apache/solr/metrics/MetricSuppliers.java   | 267 ++++++--
 .../org/apache/solr/metrics/SolrMetricManager.java |  14 +-
 .../response/transform/ChildDocTransformer.java    |  11 +-
 .../apache/solr/schema/FieldTypePluginLoader.java  |   4 +-
 .../java/org/apache/solr/schema/IndexSchema.java   |   2 +-
 .../java/org/apache/solr/schema/SchemaField.java   |  11 +
 .../java/org/apache/solr/search/CacheConfig.java   |   4 +-
 .../solr/search/CollapsingQParserPlugin.java       |   2 +
 .../src/java/org/apache/solr/search/Insanity.java  |   3 +-
 .../solr/servlet/cache/HttpCacheHeaderUtil.java    |   7 +-
 .../apache/solr/uninverting/UninvertingReader.java |   3 +-
 .../src/java/org/apache/solr/update/PeerSync.java  |  30 +-
 .../src/java/org/apache/solr/util/PackageTool.java |  12 +-
 .../src/java/org/apache/solr/util/SolrCLI.java     |   1 +
 .../solr/util/plugin/AbstractPluginLoader.java     |   2 +-
 .../apache/solr/util/plugin/MapPluginLoader.java   |   2 +-
 .../solr/util/plugin/NamedListPluginLoader.java    |   2 +-
 .../src/test-files/solr/solr-metricsconfig.xml     |   2 +-
 solr/core/src/test-files/solr/solr.xml             |   2 +
 .../apache/solr/cloud/CollectionsAPISolrJTest.java |   1 +
 .../solr/cloud/MetricsHistoryIntegrationTest.java  |   1 +
 .../MetricsHistoryWithAuthIntegrationTest.java     |   2 +-
 .../org/apache/solr/cloud/ReplaceNodeTest.java     |  39 ++
 .../test/org/apache/solr/cloud/SplitShardTest.java |   1 +
 .../apache/solr/cloud/TestBaseStatsCacheCloud.java |   1 +
 .../org/apache/solr/cloud/TestCloudRecovery.java   |   1 +
 .../org/apache/solr/cloud/TestConfigSetsAPI.java   | 190 +++++-
 .../org/apache/solr/cloud/TestTlogReplica.java     |   1 +
 .../apache/solr/cloud/rule/ImplicitSnitchTest.java | 244 -------
 .../org/apache/solr/cloud/rule/RuleEngineTest.java | 323 ---------
 .../test/org/apache/solr/cloud/rule/RulesTest.java | 328 ----------
 .../org/apache/solr/core/TestCustomStream.java     |  19 -
 .../solr/handler/admin/AdminHandlersProxyTest.java |   1 +
 .../handler/admin/MetricsHistoryHandlerTest.java   |   1 +
 .../solr/handler/admin/TestCollectionAPIs.java     |   4 -
 .../org/apache/solr/metrics/MetricsConfigTest.java |  12 +
 .../solr/metrics/MetricsDisabledCloudTest.java     |  72 ++
 .../solr/metrics/SolrMetricsIntegrationTest.java   |  13 -
 .../TestChildDocTransformerHierarchy.java          |  47 ++
 .../solr/schema/TestUseDocValuesAsStored.java      |   2 +-
 .../test/org/apache/solr/search/TestDocSet.java    |   6 +
 .../solr/search/stats/TestDefaultStatsCache.java   |   1 +
 solr/dev-docs/overseer/overseer.adoc               |   7 +-
 {lucene => solr/documentation}/build.gradle        |   6 +-
 .../images => documentation/src/assets}/solr.svg   |   0
 .../src/markdown}/SYSTEM_REQUIREMENTS.md           |   0
 .../src/markdown}/index.template.md                |   2 +-
 .../src/markdown}/online-link.template.md          |   4 +-
 solr/licenses/junit-4.12.jar.sha1                  |   1 -
 solr/licenses/junit-4.13.1.jar.sha1                |   1 +
 solr/packaging/build.gradle                        |  20 +-
 solr/server/etc/security.policy                    |   2 +-
 solr/server/solr/solr.xml                          |   2 +
 solr/site/assets/images/favicon.ico                | Bin 3262 -> 0 bytes
 .../images/quickstart-admin-ui-facet-options.png   | Bin 10646 -> 0 bytes
 .../site/assets/images/quickstart-query-screen.png | Bin 278938 -> 0 bytes
 solr/site/assets/images/quickstart-range-facet.png | Bin 21592 -> 0 bytes
 solr/site/assets/images/quickstart-solrcloud.png   | Bin 50418 -> 0 bytes
 solr/site/assets/images/quickstart-spatial.png     | Bin 229121 -> 0 bytes
 solr/site/index.xsl                                | 112 ----
 solr/site/online-link.xsl                          |  69 --
 solr/solr-ref-guide/build.gradle                   | 280 ++++----
 solr/solr-ref-guide/src/_config.yml.template       |   2 +-
 solr/solr-ref-guide/src/_includes/head.html        |   5 +-
 .../authentication-and-authorization-plugins.adoc  |   2 +-
 .../src/basic-authentication-plugin.adoc           |   2 +-
 solr/solr-ref-guide/src/blob-store-api.adoc        | 274 --------
 solr/solr-ref-guide/src/codec-factory.adoc         |   4 +-
 solr/solr-ref-guide/src/collection-management.adoc |   6 -
 .../src/common-query-parameters.adoc               |   2 +-
 solr/solr-ref-guide/src/configsets-api.adoc        |  11 +-
 solr/solr-ref-guide/src/configuration-apis.adoc    |   5 +-
 ...datadir-and-directoryfactory-in-solrconfig.adoc |   6 +-
 .../src/documents-fields-and-schema-design.adoc    |   2 +
 solr/solr-ref-guide/src/exporting-result-sets.adoc |   2 +-
 solr/solr-ref-guide/src/faceting.adoc              |   2 +-
 .../src/field-types-included-with-solr.adoc        |   6 +-
 solr/solr-ref-guide/src/filter-descriptions.adoc   |   4 +-
 .../src/hadoop-authentication-plugin.adoc          |   2 +-
 solr/solr-ref-guide/src/highlighting.adoc          |   2 +-
 .../src/implicit-requesthandlers.adoc              |  58 +-
 solr/solr-ref-guide/src/index-replication.adoc     |   2 +-
 .../src/indexconfig-in-solrconfig.adoc             |  17 +-
 .../src/indexing-nested-documents.adoc             |  10 +-
 solr/solr-ref-guide/src/language-analysis.adoc     |   2 +-
 solr/solr-ref-guide/src/learning-to-rank.adoc      |  50 +-
 solr/solr-ref-guide/src/luke-request-handler.adoc  |   2 +-
 .../src/major-changes-from-solr-5-to-solr-6.adoc   |   2 +-
 .../src/major-changes-in-solr-7.adoc               |   2 +-
 .../src/major-changes-in-solr-8.adoc               |   8 +-
 .../src/major-changes-in-solr-9.adoc               |   2 +-
 solr/solr-ref-guide/src/math-expressions.adoc      |   3 +-
 solr/solr-ref-guide/src/metrics-history.adoc       |   9 +-
 solr/solr-ref-guide/src/metrics-reporting.adoc     |  15 +
 solr/solr-ref-guide/src/other-parsers.adoc         |  26 +-
 solr/solr-ref-guide/src/other-schema-elements.adoc |   6 +-
 solr/solr-ref-guide/src/query-re-ranking.adoc      |   8 +-
 .../src/query-syntax-and-parsing.adoc              |   2 +-
 ...andlers-and-searchcomponents-in-solrconfig.adoc |   4 +-
 .../src/rule-based-replica-placement.adoc          | 177 -----
 solr/solr-ref-guide/src/schemaless-mode.adoc       |  16 +-
 .../src/searching-nested-documents.adoc            |  42 +-
 .../src/shards-and-indexing-data-in-solrcloud.adoc |   6 +-
 solr/solr-ref-guide/src/solr-field-types.adoc      |   2 +-
 solr/solr-ref-guide/src/solr-upgrade-notes.adoc    |  94 ++-
 ...solrcloud-query-routing-and-read-tolerance.adoc |   2 +-
 solr/solr-ref-guide/src/solrcloud.adoc             |   5 +-
 solr/solr-ref-guide/src/statistics.adoc            |   4 +-
 .../src/stream-decorator-reference.adoc            |   4 +-
 .../src/stream-source-reference.adoc               |   2 +-
 solr/solr-ref-guide/src/streaming-expressions.adoc |   8 +-
 .../src/taking-solr-to-production.adoc             |  10 +-
 .../src/the-query-elevation-component.adoc         |   9 +-
 solr/solr-ref-guide/src/the-terms-component.adoc   |   2 +-
 .../src/update-request-processors.adoc             | 106 +--
 .../src/updating-parts-of-documents.adoc           |  12 +-
 solr/solr-ref-guide/src/using-solrj.adoc           |  22 +-
 solr/solr-ref-guide/src/v2-api.adoc                |  10 -
 solr/solr-ref-guide/src/vectorization.adoc         |   8 +-
 .../src/zookeeper-access-control.adoc              |   6 +-
 solr/solr-ref-guide/tools/BuildNavDataFiles.java   |   1 -
 .../solr-ref-guide/tools/CheckLinksAndAnchors.java |  11 -
 .../solrj/impl/SolrClientNodeStateProvider.java    |  50 +-
 .../solrj/request/CollectionAdminRequest.java      |   7 -
 .../apache/solr/common/cloud/DocCollection.java    |   6 -
 .../apache/solr/common/params/ConfigSetParams.java |   1 +
 .../java/org/apache/solr/common}/util/DOMUtil.java |   4 +-
 .../apache/solr/common}/util/PropertiesUtil.java   |   2 +-
 .../resources/apispec/collections.Commands.json    |  16 -
 .../collections.collection.Commands.modify.json    |  16 -
 ...nfig.Commands.addRequestHandler.properties.json |   4 -
 .../apispec/core.config.Commands.generic.json      |   4 -
 .../resources/apispec/core.config.Commands.json    |  13 -
 .../solrj/impl/CloudHttp2SolrClientRetryTest.java  |   1 +
 .../solrj/impl/CloudHttp2SolrClientTest.java       |   1 +
 .../solrj/impl/CloudSolrClientRetryTest.java       |   1 +
 .../client/solrj/impl/CloudSolrClientTest.java     |   1 +
 .../org/apache/solr/common}/util/DOMUtilTest.java  |   5 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |   3 +-
 .../apache/solr/cloud/SolrCloudAuthTestCase.java   |  12 +
 solr/webapp/web/css/angular/collections.css        |   5 +
 solr/webapp/web/index.html                         |   2 +-
 .../web/js/angular/controllers/collections.js      |   7 +-
 solr/webapp/web/libs/jquery-3.4.1.min.js           |  28 -
 solr/webapp/web/libs/jquery-3.5.1.min.js           |  27 +
 solr/webapp/web/partials/collections.html          |  22 +-
 versions.lock                                      |   2 +-
 versions.props                                     |   2 +-
 326 files changed, 6381 insertions(+), 5426 deletions(-)



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

Posted by dw...@apache.org.
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;