You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2020/11/02 11:03:46 UTC

[lucene-solr] branch jira/solr-14749-api updated: SOLR-14749: Rename CustomContainerPlugins to better reflect the new role of this class. Add some dev docs about the new types of plugins.

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

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


The following commit(s) were added to refs/heads/jira/solr-14749-api by this push:
     new d63779b  SOLR-14749: Rename CustomContainerPlugins to better reflect the new role of this class. Add some dev docs about the new types of plugins.
d63779b is described below

commit d63779b067463c1b966f7fd3f02fb595c34423f2
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Mon Nov 2 12:02:53 2020 +0100

    SOLR-14749: Rename CustomContainerPlugins to better reflect the new role of this class.
    Add some dev docs about the new types of plugins.
---
 ...rPlugins.java => ContainerPluginsRegistry.java} |  4 +-
 .../solr/cluster/events/ClusterEventProducer.java  | 38 +---------
 .../apache/solr/cluster/events/NoOpProducer.java   | 54 +++++++++++++++
 .../solr/core/ClusterEventProducerFactory.java     | 30 ++++----
 .../org/apache/solr/core/ClusterSingletons.java    | 16 ++---
 .../java/org/apache/solr/core/CoreContainer.java   | 18 ++---
 .../solr/handler/admin/ContainerPluginsApi.java    |  4 +-
 solr/dev-docs/plugins/container-plugins.adoc       | 81 ++++++++++++++++++++++
 8 files changed, 172 insertions(+), 73 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java b/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
similarity index 98%
rename from solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
rename to solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
index 91b0452..9154ae4 100644
--- a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
+++ b/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
@@ -65,7 +65,7 @@ import static org.apache.solr.common.util.Utils.makeMap;
  * for additional functionality by {@link PluginRegistryListener}-s registered with
  * this class.</p>
  */
-public class CustomContainerPlugins implements ClusterPropertiesListener, MapWriter {
+public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapWriter {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private final ObjectMapper mapper = SolrJacksonAnnotationInspector.createObjectMapper();
@@ -90,7 +90,7 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
     listeners.remove(listener);
   }
 
-  public CustomContainerPlugins(CoreContainer coreContainer, ApiBag apiBag) {
+  public ContainerPluginsRegistry(CoreContainer coreContainer, ApiBag apiBag) {
     this.coreContainer = coreContainer;
     this.containerApiBag = apiBag;
   }
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
index a39e751..f87ce7c 100644
--- a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
+++ b/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
@@ -24,7 +24,7 @@ import org.apache.solr.cloud.ClusterSingleton;
 public interface ClusterEventProducer extends ClusterSingleton {
 
   /** Unique name for the registration of a plugin-based implementation. */
-  String PLUGIN_NAME = "clusterEventProducer";
+  String PLUGIN_NAME = "cluster-event-producer";
 
   @Override
   default String getName() {
@@ -57,40 +57,4 @@ public interface ClusterEventProducer extends ClusterSingleton {
    */
   void unregisterListener(ClusterEventListener listener, ClusterEvent.EventType... eventTypes);
 
-  /**
-   * No-op implementation of {@link ClusterEventProducer}. This implementation is always in
-   * RUNNING state.
-   */
-  final class NoOpProducer implements ClusterEventProducer {
-
-    @Override
-    public void registerListener(ClusterEventListener listener, ClusterEvent.EventType... eventTypes) {
-      // no-op
-    }
-
-    @Override
-    public void unregisterListener(ClusterEventListener listener, ClusterEvent.EventType... eventTypes) {
-      // no-op
-    }
-
-    @Override
-    public String getName() {
-      return ClusterEventProducer.PLUGIN_NAME;
-    }
-
-    @Override
-    public void start() throws Exception {
-      // no-op
-    }
-
-    @Override
-    public State getState() {
-      return State.RUNNING;
-    }
-
-    @Override
-    public void stop() {
-      // no-op
-    }
-  }
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/NoOpProducer.java b/solr/core/src/java/org/apache/solr/cluster/events/NoOpProducer.java
new file mode 100644
index 0000000..8632895
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cluster/events/NoOpProducer.java
@@ -0,0 +1,54 @@
+/*
+ * 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;
+
+/**
+ * No-op implementation of {@link ClusterEventProducer}. This implementation is always in
+ * RUNNING state.
+ */
+public final class NoOpProducer implements ClusterEventProducer {
+
+  @Override
+  public void registerListener(ClusterEventListener listener, ClusterEvent.EventType... eventTypes) {
+    // no-op
+  }
+
+  @Override
+  public void unregisterListener(ClusterEventListener listener, ClusterEvent.EventType... eventTypes) {
+    // no-op
+  }
+
+  @Override
+  public String getName() {
+    return ClusterEventProducer.PLUGIN_NAME;
+  }
+
+  @Override
+  public void start() throws Exception {
+    // no-op
+  }
+
+  @Override
+  public State getState() {
+    return State.RUNNING;
+  }
+
+  @Override
+  public void stop() {
+    // no-op
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/core/ClusterEventProducerFactory.java b/solr/core/src/java/org/apache/solr/core/ClusterEventProducerFactory.java
index b0b658e..0a38414 100644
--- a/solr/core/src/java/org/apache/solr/core/ClusterEventProducerFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/ClusterEventProducerFactory.java
@@ -1,6 +1,6 @@
 package org.apache.solr.core;
 
-import org.apache.solr.api.CustomContainerPlugins;
+import org.apache.solr.api.ContainerPluginsRegistry;
 import org.apache.solr.cluster.events.ClusterEvent;
 import org.apache.solr.cluster.events.ClusterEventListener;
 import org.apache.solr.cluster.events.ClusterEventProducer;
@@ -18,15 +18,15 @@ import java.util.Set;
  */
 public class ClusterEventProducerFactory implements ClusterEventProducer {
   private Map<ClusterEvent.EventType, Set<ClusterEventListener>> initialListeners = new HashMap<>();
-  private CustomContainerPlugins.PluginRegistryListener initialPluginListener;
+  private ContainerPluginsRegistry.PluginRegistryListener initialPluginListener;
   private final CoreContainer cc;
   private boolean created = false;
 
   public ClusterEventProducerFactory(CoreContainer cc) {
     this.cc = cc;
-    initialPluginListener = new CustomContainerPlugins.PluginRegistryListener() {
+    initialPluginListener = new ContainerPluginsRegistry.PluginRegistryListener() {
       @Override
-      public void added(CustomContainerPlugins.ApiInfo plugin) {
+      public void added(ContainerPluginsRegistry.ApiInfo plugin) {
         if (plugin == null || plugin.getInstance() == null) {
           return;
         }
@@ -37,7 +37,7 @@ public class ClusterEventProducerFactory implements ClusterEventProducer {
       }
 
       @Override
-      public void deleted(CustomContainerPlugins.ApiInfo plugin) {
+      public void deleted(ContainerPluginsRegistry.ApiInfo plugin) {
         if (plugin == null || plugin.getInstance() == null) {
           return;
         }
@@ -48,7 +48,7 @@ public class ClusterEventProducerFactory implements ClusterEventProducer {
       }
 
       @Override
-      public void modified(CustomContainerPlugins.ApiInfo old, CustomContainerPlugins.ApiInfo replacement) {
+      public void modified(ContainerPluginsRegistry.ApiInfo old, ContainerPluginsRegistry.ApiInfo replacement) {
         added(replacement);
         deleted(old);
       }
@@ -60,7 +60,7 @@ public class ClusterEventProducerFactory implements ClusterEventProducer {
    * freshly loaded listener plugins before the final cluster event producer is created.
    * @return initial listener
    */
-  public CustomContainerPlugins.PluginRegistryListener getPluginRegistryListener() {
+  public ContainerPluginsRegistry.PluginRegistryListener getPluginRegistryListener() {
     return initialPluginListener;
   }
 
@@ -68,17 +68,17 @@ public class ClusterEventProducerFactory implements ClusterEventProducer {
    * Create a {@link ClusterEventProducer} based on the current plugin configurations.
    * <p>NOTE: this method can only be called once because it has side-effects, such as
    * transferring the initially collected listeners to the resulting producer's instance, and
-   * installing a {@link org.apache.solr.api.CustomContainerPlugins.PluginRegistryListener}.
+   * installing a {@link ContainerPluginsRegistry.PluginRegistryListener}.
    * Calling this method more than once will result in an exception.</p>
    * @param plugins current plugin configurations
    * @return configured instance of cluster event producer (with side-effects, see above)
    */
-  public ClusterEventProducer create(CustomContainerPlugins plugins) {
+  public ClusterEventProducer create(ContainerPluginsRegistry plugins) {
     if (created) {
       throw new RuntimeException("this factory can be called only once!");
     }
     final ClusterEventProducer clusterEventProducer;
-    CustomContainerPlugins.ApiInfo clusterEventProducerInfo = plugins.getPlugin(ClusterEventProducer.PLUGIN_NAME);
+    ContainerPluginsRegistry.ApiInfo clusterEventProducerInfo = plugins.getPlugin(ClusterEventProducer.PLUGIN_NAME);
     if (clusterEventProducerInfo != null) {
       // the listener in ClusterSingletons already registered it
       clusterEventProducer = (ClusterEventProducer) clusterEventProducerInfo.getInstance();
@@ -91,9 +91,9 @@ public class ClusterEventProducerFactory implements ClusterEventProducer {
     // transfer those listeners that were already registered to the initial impl
     transferListeners(clusterEventProducer, plugins);
     // install plugin registry listener
-    CustomContainerPlugins.PluginRegistryListener pluginListener = new CustomContainerPlugins.PluginRegistryListener() {
+    ContainerPluginsRegistry.PluginRegistryListener pluginListener = new ContainerPluginsRegistry.PluginRegistryListener() {
       @Override
-      public void added(CustomContainerPlugins.ApiInfo plugin) {
+      public void added(ContainerPluginsRegistry.ApiInfo plugin) {
         if (plugin == null || plugin.getInstance() == null) {
           return;
         }
@@ -105,7 +105,7 @@ public class ClusterEventProducerFactory implements ClusterEventProducer {
       }
 
       @Override
-      public void deleted(CustomContainerPlugins.ApiInfo plugin) {
+      public void deleted(ContainerPluginsRegistry.ApiInfo plugin) {
         if (plugin == null || plugin.getInstance() == null) {
           return;
         }
@@ -117,7 +117,7 @@ public class ClusterEventProducerFactory implements ClusterEventProducer {
       }
 
       @Override
-      public void modified(CustomContainerPlugins.ApiInfo old, CustomContainerPlugins.ApiInfo replacement) {
+      public void modified(ContainerPluginsRegistry.ApiInfo old, ContainerPluginsRegistry.ApiInfo replacement) {
         added(replacement);
         deleted(old);
       }
@@ -127,7 +127,7 @@ public class ClusterEventProducerFactory implements ClusterEventProducer {
     return clusterEventProducer;
   }
 
-  private void transferListeners(ClusterEventProducer target, CustomContainerPlugins plugins) {
+  private void transferListeners(ClusterEventProducer target, ContainerPluginsRegistry plugins) {
     // stop capturing listener plugins
     plugins.unregisterListener(initialPluginListener);
     // transfer listeners that are already registered
diff --git a/solr/core/src/java/org/apache/solr/core/ClusterSingletons.java b/solr/core/src/java/org/apache/solr/core/ClusterSingletons.java
index 0487540..82564e6 100644
--- a/solr/core/src/java/org/apache/solr/core/ClusterSingletons.java
+++ b/solr/core/src/java/org/apache/solr/core/ClusterSingletons.java
@@ -17,7 +17,7 @@
 
 package org.apache.solr.core;
 
-import org.apache.solr.api.CustomContainerPlugins;
+import org.apache.solr.api.ContainerPluginsRegistry;
 import org.apache.solr.cloud.ClusterSingleton;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -33,7 +33,7 @@ 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}.
+ * to track the changes in loaded plugins in {@link ContainerPluginsRegistry}.
  */
 public class ClusterSingletons {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -41,7 +41,7 @@ public class ClusterSingletons {
   private final Map<String, ClusterSingleton> singletonMap = new ConcurrentHashMap<>();
   private final Supplier<Boolean> runSingletons;
   private final Consumer<Runnable> asyncRunner;
-  private final CustomContainerPlugins.PluginRegistryListener pluginListener;
+  private final ContainerPluginsRegistry.PluginRegistryListener pluginListener;
 
   public static final int DEFAULT_WAIT_TIMEOUT_SEC = 60;
 
@@ -60,9 +60,9 @@ public class ClusterSingletons {
     this.runSingletons = runSingletons;
     this.asyncRunner = asyncRunner;
     // create plugin registry listener
-    pluginListener = new CustomContainerPlugins.PluginRegistryListener() {
+    pluginListener = new ContainerPluginsRegistry.PluginRegistryListener() {
       @Override
-      public void added(CustomContainerPlugins.ApiInfo plugin) {
+      public void added(ContainerPluginsRegistry.ApiInfo plugin) {
         if (plugin == null || plugin.getInstance() == null) {
           return;
         }
@@ -83,7 +83,7 @@ public class ClusterSingletons {
       }
 
       @Override
-      public void deleted(CustomContainerPlugins.ApiInfo plugin) {
+      public void deleted(ContainerPluginsRegistry.ApiInfo plugin) {
         if (plugin == null || plugin.getInstance() == null) {
           return;
         }
@@ -96,14 +96,14 @@ public class ClusterSingletons {
       }
 
       @Override
-      public void modified(CustomContainerPlugins.ApiInfo old, CustomContainerPlugins.ApiInfo replacement) {
+      public void modified(ContainerPluginsRegistry.ApiInfo old, ContainerPluginsRegistry.ApiInfo replacement) {
         added(replacement);
         deleted(old);
       }
     };
   }
 
-  public CustomContainerPlugins.PluginRegistryListener getPluginRegistryListener() {
+  public ContainerPluginsRegistry.PluginRegistryListener getPluginRegistryListener() {
     return pluginListener;
   }
 
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 137e26a..78cdd09 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -56,7 +56,7 @@ import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.store.Directory;
-import org.apache.solr.api.CustomContainerPlugins;
+import org.apache.solr.api.ContainerPluginsRegistry;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
@@ -178,7 +178,7 @@ public class CoreContainer {
    */
   public final Supplier<SolrZkClient> zkClientSupplier = () -> getZkController().getZkClient();
 
-  private final CustomContainerPlugins customContainerPlugins =  new CustomContainerPlugins(this, containerHandlers.getApiBag());
+  private final ContainerPluginsRegistry containerPluginsRegistry =  new ContainerPluginsRegistry(this, containerHandlers.getApiBag());
 
   protected final Map<String, CoreLoadFailure> coreInitFailures = new ConcurrentHashMap<>();
 
@@ -678,8 +678,8 @@ public class CoreContainer {
     ClusterEventProducerFactory clusterEventProducerFactory = new ClusterEventProducerFactory(this);
     clusterEventProducer = clusterEventProducerFactory;
 
-    customContainerPlugins.registerListener(clusterSingletons.getPluginRegistryListener());
-    customContainerPlugins.registerListener(clusterEventProducerFactory.getPluginRegistryListener());
+    containerPluginsRegistry.registerListener(clusterSingletons.getPluginRegistryListener());
+    containerPluginsRegistry.registerListener(clusterEventProducerFactory.getPluginRegistryListener());
 
     packageStoreAPI = new PackageStoreAPI(this);
     containerHandlers.getApiBag().registerObject(packageStoreAPI.readAPI);
@@ -889,14 +889,14 @@ public class CoreContainer {
     }
 
     if (isZooKeeperAware()) {
-      customContainerPlugins.refresh();
-      getZkController().zkStateReader.registerClusterPropertiesListener(customContainerPlugins);
+      containerPluginsRegistry.refresh();
+      getZkController().zkStateReader.registerClusterPropertiesListener(containerPluginsRegistry);
       ContainerPluginsApi containerPluginsApi = new ContainerPluginsApi(this);
       containerHandlers.getApiBag().registerObject(containerPluginsApi.readAPI);
       containerHandlers.getApiBag().registerObject(containerPluginsApi.editAPI);
 
       // create target ClusterEventProducer (possibly from plugins)
-      clusterEventProducer = clusterEventProducerFactory.create(customContainerPlugins);
+      clusterEventProducer = clusterEventProducerFactory.create(containerPluginsRegistry);
 
       // init ClusterSingleton-s
 
@@ -2130,8 +2130,8 @@ public class CoreContainer {
     return tragicException != null;
   }
 
-  public CustomContainerPlugins getCustomContainerPlugins(){
-    return customContainerPlugins;
+  public ContainerPluginsRegistry getContainerPluginsRegistry() {
+    return containerPluginsRegistry;
   }
 
   public ClusterSingletons getClusterSingletons() {
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 f6af915..aff5484 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
@@ -28,7 +28,7 @@ import java.util.function.Supplier;
 
 import org.apache.solr.api.AnnotatedApi;
 import org.apache.solr.api.Command;
-import org.apache.solr.api.CustomContainerPlugins;
+import org.apache.solr.api.ContainerPluginsRegistry;
 import org.apache.solr.api.EndPoint;
 import org.apache.solr.api.PayloadObj;
 import org.apache.solr.client.solrj.SolrRequest.METHOD;
@@ -137,7 +137,7 @@ public class ContainerPluginsApi {
       }
     }
     List<String> errs = new ArrayList<>();
-    CustomContainerPlugins.ApiInfo apiInfo = coreContainer.getCustomContainerPlugins().createInfo(info, errs);
+    ContainerPluginsRegistry.ApiInfo apiInfo = coreContainer.getContainerPluginsRegistry().createInfo(info, errs);
     if (!errs.isEmpty()) {
       for (String err : errs) payload.addError(err);
       return;
diff --git a/solr/dev-docs/plugins/container-plugins.adoc b/solr/dev-docs/plugins/container-plugins.adoc
new file mode 100644
index 0000000..b10ba4b
--- /dev/null
+++ b/solr/dev-docs/plugins/container-plugins.adoc
@@ -0,0 +1,81 @@
+= Container Plugins
+:toc: macro
+:toclevels: 3
+
+|===
+| Initial version released| 2020, November 2|Andrzej BiaƂecki
+|===
+
+toc::[]
+
+== Container plugins subsystem
+Container plugins are pluggable components that are defined and instantiated at the
+`CoreContainer` (node) level. The components usually provide an admin-level API for
+additional functionality at the Solr node level.
+
+=== Plugin configurations
+Plugin configurations are maintained in ZooKeeper in the `/clusterprops.json` file, under
+the `plugin` entry. The configuration is a JSON map where keys are the plugin names, and
+values are serialized `org.apache.solr.client.solrj.request.beans.PluginMeta` beans.
+
+== Types of container plugins
+
+=== Plugin life-cycle
+Plugin instances are loaded and initialized when Solr's `CoreContainer` is first created.
+
+Then on each update of the plugin configurations the existing plugin configs are compared
+with the new configs, and plugin instances are respectively created, removed, or
+replaced (i.e. removed and added using the new configuration).
+
+If a plugin implementation class has a constructor that accepts an instance of
+`CoreContainer` then it is instantiated using this constructor, and the current instance
+of `CoreContainer` is passed as argument.
+
+=== PluginRegistryListener
+Components that need to be aware of changes in plugin configurations or registration can
+implement `org.apache.solr.api.ContainerPluginsRegistry.PluginRegistryListener` and register
+it with the instance of registry available from `coreContainer.getContainerPluginsRegistry()`.
+
+=== ClusterSingleton plugins
+Plugins that implement `ClusterSingleton` interface are instantiated on each
+Solr node. However, their start/stop life-cycle, as defined in the interface,
+is controlled in such a way that only a single running instance of the plugin
+is present in the cluster at any time.
+
+(Currently this is implemented by re-using the Overseer leader election, so all
+`ClusterSingleton`-s that are in the RUNNING state execute on the Overseer leader).
+
+Any plugin type can implement this interface to indicate to Solr that
+it requires this cluster singleton behavior.
+
+// explain plugins that register Api-s
+// explain plugins that don't implement any Api
+=== ClusterEventProducer plugin
+In order to support the generation of cluster-level events an implementation of
+`ClusterEventProducer` is created on each Solr node. This component is also a
+`ClusterSingleton`, which means that only one active instance is present in the
+cluster at any time.
+
+If no plugin configuration is specified then the default implementation
+`org.apache.solr.cluster.events.impl.DefaultClusterEventProducer` is used. A no-op
+implementation is also available in `org.apache.solr.cluster.events.NoOpProducer`.
+
+=== ClusterEventListener plugins
+Plugins that implement the `ClusterEventListener` interface will be automatically
+registered with the instance of `ClusterEventProducer`.
+
+== Plugin management API
+
+== Predefined plugin names
+
+Plugins with these names are used in specific parts of Solr. Their names are reserved
+and cannot be used for other plugin types:
+
+// XXX uncomment when we move the config to plugins
+//`placement-plugin`::
+//plugin that implements `PlacementPlugin` interface. This type of plugin
+//determines the replica placement strategy in the cluster.
+
+`cluster-event-producer`::
+plugin that implements `ClusterEventProducer` interface. This type of plugin
+is used for generating cluster-level events.
\ No newline at end of file