You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ds...@apache.org on 2024/01/17 23:19:25 UTC

(solr) branch main updated: SOLR-17096: Cluster Singleton plugin support in solr.xml (#2126)

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

dsmiley pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/main by this push:
     new 1de6d081009 SOLR-17096: Cluster Singleton plugin support in solr.xml (#2126)
1de6d081009 is described below

commit 1de6d081009d835116df89df3cc2c8d66c1efff4
Author: pjmcarthur <92...@users.noreply.github.com>
AuthorDate: Wed Jan 17 15:19:19 2024 -0800

    SOLR-17096: Cluster Singleton plugin support in solr.xml (#2126)
    
    * Generalize to all "cluster plugins", albeit only supporting <clusterSingleton> and <replicaPlacementFactory> at this time.
    * Introduced new `solr.cluster.plugin.edit.enabled` boolean setting to disable /cluster/plugin mutability, required for solr.xml's use of those plugins.
    
    Co-authored-by: Paul McArthur <pm...@proton.me>
---
 solr/CHANGES.txt                                   |   2 +
 .../org/apache/solr/api/ClusterPluginsSource.java  |  83 ++++++++
 .../apache/solr/api/ContainerPluginsRegistry.java  |  14 +-
 .../solr/api/NodeConfigClusterPluginsSource.java   | 106 ++++++++++
 .../apache/solr/api/ZkClusterPluginsSource.java    | 114 +++++++++++
 .../impl/PlacementPluginFactoryLoader.java         |  42 +---
 .../java/org/apache/solr/core/CoreContainer.java   |  19 +-
 .../src/java/org/apache/solr/core/NodeConfig.java  |  44 +++--
 .../java/org/apache/solr/core/SolrXmlConfig.java   |  68 ++++++-
 .../org/apache/solr/handler/SolrConfigHandler.java |   5 +-
 .../solr/handler/admin/ContainerPluginsApi.java    |  80 ++------
 solr/core/src/test-files/solr/solr-50-all.xml      |   3 +
 .../api/NodeConfigClusterPluginsSourceTest.java    | 219 +++++++++++++++++++++
 .../impl/NodeConfigPlacementPluginTest.java        |  63 ++++++
 .../impl/PlacementPluginIntegrationTest.java       |  19 --
 .../src/test/org/apache/solr/core/TestSolrXml.java | 126 +++++++++++-
 .../configuration-guide/pages/cluster-plugins.adoc |   4 +-
 .../pages/configuring-solr-xml.adoc                |  19 ++
 18 files changed, 870 insertions(+), 160 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 808ca7a4d73..24d05d6ae3d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -23,6 +23,8 @@ Improvements
 
 * SOLR-17077: When a shard rejoins leader election, leave previous election only once to save unneeded calls to Zookeeper. (Pierre Salagnac)
 
+* SOLR-17096: solr.xml now supports declaring clusterSingleton plugins (Paul McArthur, David Smiley)
+
 Optimizations
 ---------------------
 (No changes)
diff --git a/solr/core/src/java/org/apache/solr/api/ClusterPluginsSource.java b/solr/core/src/java/org/apache/solr/api/ClusterPluginsSource.java
new file mode 100644
index 00000000000..cb3e7af4dc9
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/ClusterPluginsSource.java
@@ -0,0 +1,83 @@
+/*
+ * 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.api;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Function;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.common.util.EnvUtils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.handler.admin.ContainerPluginsApi;
+
+/** A source for Cluster Plugin configurations */
+public interface ClusterPluginsSource {
+
+  /**
+   * Resolves the name of the class that will be used to provide cluster plugins.
+   *
+   * @return The name of the class to use as the {@link ClusterPluginsSource}
+   */
+  public static String resolveClassName() {
+    return EnvUtils.getPropAsBool(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, true)
+        ? ZkClusterPluginsSource.class.getName()
+        : NodeConfigClusterPluginsSource.class.getName();
+  }
+
+  public static ClusterPluginsSource loadClusterPluginsSource(
+      CoreContainer cc, SolrResourceLoader loader) {
+    return loader.newInstance(
+        resolveClassName(),
+        ClusterPluginsSource.class,
+        new String[0],
+        new Class<?>[] {CoreContainer.class},
+        new Object[] {cc});
+  }
+
+  /**
+   * Get the Read Api for this plugin source
+   *
+   * @return A {@link ContainerPluginsApi} Read Api for this plugin source
+   */
+  ContainerPluginsApi.Read getReadApi();
+
+  /**
+   * Get the Edit Api for this plugin source, if it supports edit operations
+   *
+   * @return A {@link ContainerPluginsApi} Edit Api for this plugin source, or null if the plugin
+   *     source does not support editing the plugin configs
+   */
+  ContainerPluginsApi.Edit getEditApi();
+
+  /**
+   * Get a map of cluster plugin configurations from this source, where keys are plugin names and
+   * values are {@link PluginMeta} plugin metadata.
+   *
+   * @return An immutable map of plugin configurations
+   */
+  Map<String, Object> plugins() throws IOException;
+
+  /**
+   * Persist the updated set of plugin configs
+   *
+   * @param modifier A function providing the map of plugin configs to be persisted
+   */
+  void persistPlugins(Function<Map<String, Object>, Map<String, Object>> modifier)
+      throws IOException;
+}
diff --git a/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java b/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
index fbfa6719e8e..4573a6e550e 100644
--- a/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
+++ b/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
@@ -39,7 +39,6 @@ import java.util.Objects;
 import java.util.Optional;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.Phaser;
-import java.util.function.Supplier;
 import org.apache.lucene.util.ResourceLoaderAware;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.request.beans.PluginMeta;
@@ -55,7 +54,6 @@ import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.PluginInfo;
-import org.apache.solr.handler.admin.ContainerPluginsApi;
 import org.apache.solr.pkg.SolrPackageLoader;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
@@ -66,7 +64,7 @@ import org.slf4j.LoggerFactory;
 /**
  * This class manages the container-level plugins and their Api-s. It is responsible for adding /
  * removing / replacing the plugins according to the updated configuration obtained from {@link
- * ContainerPluginsApi#plugins(Supplier)}.
+ * ClusterPluginsSource#plugins()}.
  *
  * <p>Plugins instantiated by this class may implement zero or more {@link Api}-s, which are then
  * registered in the CoreContainer {@link ApiBag}. They may be also post-processed for additional
@@ -75,6 +73,8 @@ import org.slf4j.LoggerFactory;
 public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapWriter, Closeable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  public static final String CLUSTER_PLUGIN_EDIT_ENABLED = "solr.cluster.plugin.edit.enabled";
+
   private static final ObjectMapper mapper =
       SolrJacksonAnnotationInspector.createObjectMapper()
           .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false)
@@ -85,6 +85,8 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
   private final CoreContainer coreContainer;
   private final ApiBag containerApiBag;
 
+  private final ClusterPluginsSource pluginsSource;
+
   private final Map<String, ApiInfo> currentPlugins = new HashMap<>();
 
   private Phaser phaser;
@@ -117,9 +119,11 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
     listeners.remove(listener);
   }
 
-  public ContainerPluginsRegistry(CoreContainer coreContainer, ApiBag apiBag) {
+  public ContainerPluginsRegistry(
+      CoreContainer coreContainer, ApiBag apiBag, ClusterPluginsSource pluginsSource) {
     this.coreContainer = coreContainer;
     this.containerApiBag = apiBag;
+    this.pluginsSource = pluginsSource;
   }
 
   @Override
@@ -171,7 +175,7 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
   public synchronized void refresh() {
     Map<String, Object> pluginInfos;
     try {
-      pluginInfos = ContainerPluginsApi.plugins(coreContainer.zkClientSupplier);
+      pluginInfos = pluginsSource.plugins();
     } catch (IOException e) {
       log.error("Could not read plugins data", e);
       return;
diff --git a/solr/core/src/java/org/apache/solr/api/NodeConfigClusterPluginsSource.java b/solr/core/src/java/org/apache/solr/api/NodeConfigClusterPluginsSource.java
new file mode 100644
index 00000000000..4282cce69b2
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/NodeConfigClusterPluginsSource.java
@@ -0,0 +1,106 @@
+/*
+ * 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.api;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Function;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.handler.admin.ContainerPluginsApi;
+
+/**
+ * Plugin configurations that are defined in solr.xml. This supports immutable deployments, and the
+ * /cluster/plugin Edit APIs are not available
+ */
+public class NodeConfigClusterPluginsSource implements ClusterPluginsSource {
+
+  private final Map<String, Object> plugins;
+
+  private final ContainerPluginsApi api;
+
+  public NodeConfigClusterPluginsSource(final CoreContainer cc) {
+    api = new ContainerPluginsApi(cc, this);
+    plugins = Map.copyOf(readPlugins(cc.getNodeConfig()));
+  }
+
+  @Override
+  public ContainerPluginsApi.Read getReadApi() {
+    return api.readAPI;
+  }
+
+  @Override
+  public ContainerPluginsApi.Edit getEditApi() {
+    return null;
+  }
+
+  @Override
+  public Map<String, Object> plugins() throws IOException {
+    return plugins;
+  }
+
+  /**
+   * This method should never be invoked because the Edit Apis are not made available by the plugin
+   *
+   * @throws UnsupportedOperationException always
+   */
+  @Override
+  public void persistPlugins(Function<Map<String, Object>, Map<String, Object>> modifier) {
+    throw new UnsupportedOperationException(
+        "The NodeConfigContainerPluginsSource does not support updates to plugin configurations");
+  }
+
+  private static Map<String, Object> readPlugins(final NodeConfig cfg) {
+    Map<String, Object> pluginInfos = new HashMap<>();
+    PluginInfo[] clusterPlugins = cfg.getClusterPlugins();
+    if (clusterPlugins != null) {
+      for (PluginInfo p : clusterPlugins) {
+        Map<String, Object> pluginMap = new HashMap<>();
+        final String pluginName = getPluginName(p);
+        pluginMap.put("name", pluginName);
+        pluginMap.put("class", p.className);
+
+        if (p.initArgs.size() > 0) {
+          Map<String, Object> config = p.initArgs.toMap(new HashMap<>());
+          pluginMap.put("config", config);
+        }
+
+        pluginInfos.put(pluginName, pluginMap);
+      }
+    }
+    return pluginInfos;
+  }
+
+  /**
+   * Get the correct name for a plugin. Custom plugins must have a name set already, but built-in
+   * plugins may omit the name in solr.xml and require inference here
+   */
+  private static String getPluginName(final PluginInfo pluginInfo) {
+
+    if (pluginInfo.name == null) {
+      if (pluginInfo.type.equals("replicaPlacementFactory")) {
+        return PlacementPluginFactory.PLUGIN_NAME;
+      }
+    }
+
+    return pluginInfo.name;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/api/ZkClusterPluginsSource.java b/solr/core/src/java/org/apache/solr/api/ZkClusterPluginsSource.java
new file mode 100644
index 00000000000..6d2972d29ba
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/ZkClusterPluginsSource.java
@@ -0,0 +1,114 @@
+/*
+ * 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.api;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.ContainerPluginsApi;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * The plugin configurations are stored and retrieved from the ZooKeeper cluster properties, stored
+ * at the {@link ZkStateReader#CONTAINER_PLUGINS} location This supports mutable configurations, and
+ * management via the /cluster/plugin APIs
+ */
+public class ZkClusterPluginsSource implements ClusterPluginsSource {
+
+  private final Supplier<SolrZkClient> zkClientSupplier;
+
+  private final ContainerPluginsApi api;
+
+  public ZkClusterPluginsSource(CoreContainer coreContainer) {
+    this.zkClientSupplier = coreContainer.zkClientSupplier;
+    this.api = new ContainerPluginsApi(coreContainer, this);
+  }
+
+  @Override
+  public ContainerPluginsApi.Read getReadApi() {
+    return api.readAPI;
+  }
+
+  @Override
+  public ContainerPluginsApi.Edit getEditApi() {
+    return api.editAPI;
+  }
+
+  /**
+   * Retrieve the current plugin configurations.
+   *
+   * @return current plugin configurations, where keys are plugin names and values are {@link
+   *     PluginMeta} plugin metadata.
+   * @throws IOException on IO errors
+   */
+  @Override
+  @SuppressWarnings("unchecked")
+  public Map<String, Object> plugins() throws IOException {
+    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.copyOf(
+          (Map<String, Object>)
+              clusterPropsJson.computeIfAbsent(
+                  ZkStateReader.CONTAINER_PLUGINS, o -> new LinkedHashMap<>()));
+    } catch (KeeperException.NoNodeException e) {
+      return new LinkedHashMap<>();
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error reading cluster property", SolrZkClient.checkInterrupted(e));
+    }
+  }
+
+  @Override
+  public void persistPlugins(Function<Map<String, Object>, Map<String, Object>> modifier)
+      throws IOException {
+    try {
+      zkClientSupplier
+          .get()
+          .atomicUpdate(
+              ZkStateReader.CLUSTER_PROPS,
+              bytes -> {
+                @SuppressWarnings("unchecked")
+                Map<String, Object> rawJson =
+                    bytes == null
+                        ? new LinkedHashMap<>()
+                        : (Map<String, Object>) Utils.fromJSON(bytes);
+                @SuppressWarnings("unchecked")
+                Map<String, Object> pluginsModified =
+                    modifier.apply(
+                        (Map<String, Object>)
+                            rawJson.computeIfAbsent(
+                                ZkStateReader.CONTAINER_PLUGINS, o -> new LinkedHashMap<>()));
+                if (pluginsModified == null) return null;
+                rawJson.put(ZkStateReader.CONTAINER_PLUGINS, pluginsModified);
+                return Utils.toJSON(rawJson);
+              });
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error reading cluster property", SolrZkClient.checkInterrupted(e));
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java
index 45d10012028..4df4616648a 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java
@@ -18,11 +18,8 @@
 package org.apache.solr.cluster.placement.impl;
 
 import com.google.common.annotations.VisibleForTesting;
-import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import java.util.HashMap;
 import java.util.Locale;
-import java.util.Map;
 import org.apache.solr.api.ContainerPluginsRegistry;
 import org.apache.solr.client.solrj.request.beans.PluginMeta;
 import org.apache.solr.cluster.placement.PlacementPluginConfig;
@@ -32,9 +29,6 @@ import org.apache.solr.cluster.placement.plugins.MinimizeCoresPlacementFactory;
 import org.apache.solr.cluster.placement.plugins.RandomPlacementFactory;
 import org.apache.solr.cluster.placement.plugins.SimplePlacementFactory;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.core.NodeConfig;
-import org.apache.solr.core.PluginInfo;
-import org.apache.solr.core.SolrResourceLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -105,39 +99,9 @@ public class PlacementPluginFactoryLoader {
     plugins.registerListener(pluginListener);
   }
 
-  /** Returns the default {@link PlacementPluginFactory} configured in solr.xml. */
-  public static PlacementPluginFactory<?> getDefaultPlacementPluginFactory(
-      NodeConfig nodeConfig, SolrResourceLoader loader) {
-    PluginInfo pluginInfo = nodeConfig.getReplicaPlacementFactoryConfig();
-    if (null != pluginInfo) {
-      return getPlacementPluginFactory(pluginInfo, loader);
-    } else {
-      return getDefaultPlacementPluginFactory();
-    }
-  }
-
-  private static PlacementPluginFactory<?> getPlacementPluginFactory(
-      PluginInfo pluginInfo, SolrResourceLoader loader) {
-    // Load placement plugin factory from solr.xml.
-    PlacementPluginFactory<?> placementPluginFactory =
-        loader.newInstance(pluginInfo, PlacementPluginFactory.class, false);
-    if (null != pluginInfo.initArgs) {
-      Map<String, Object> config = new HashMap<>();
-      pluginInfo.initArgs.toMap(config);
-      try {
-        ContainerPluginsRegistry.configure(placementPluginFactory, config, null);
-      } catch (IOException e) {
-        throw new SolrException(
-            SolrException.ErrorCode.SERVER_ERROR,
-            "Invalid " + pluginInfo.type + " configuration",
-            e);
-      }
-    }
-    return placementPluginFactory;
-  }
-
-  private static PlacementPluginFactory<?> getDefaultPlacementPluginFactory() {
-    // Otherwise use the default provided by system properties.
+  /** Returns the default {@link PlacementPluginFactory} */
+  public static PlacementPluginFactory<?> getDefaultPlacementPluginFactory() {
+    // Use the default provided by system properties.
     String defaultPluginId = System.getProperty(PLACEMENTPLUGIN_DEFAULT_SYSPROP);
     if (defaultPluginId != null) {
       log.info(
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 ce5e05b2657..485df16dce8 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -61,6 +61,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.ClusterPluginsSource;
 import org.apache.solr.api.ContainerPluginsRegistry;
 import org.apache.solr.api.JerseyResource;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
@@ -108,7 +109,6 @@ import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.SnapShooter;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
-import org.apache.solr.handler.admin.ContainerPluginsApi;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.admin.CoreAdminHandler.CoreAdminOp;
 import org.apache.solr.handler.admin.HealthCheckHandler;
@@ -203,8 +203,7 @@ public class CoreContainer {
   /** Minimize exposure to CoreContainer. Mostly only ZK interface is required */
   public final Supplier<SolrZkClient> zkClientSupplier = () -> getZkController().getZkClient();
 
-  private final ContainerPluginsRegistry containerPluginsRegistry =
-      new ContainerPluginsRegistry(this, containerHandlers.getApiBag());
+  private ContainerPluginsRegistry containerPluginsRegistry;
 
   protected final Map<String, CoreLoadFailure> coreInitFailures = new ConcurrentHashMap<>();
 
@@ -728,7 +727,7 @@ public class CoreContainer {
   }
 
   private void registerV2ApiIfEnabled(Object apiObject) {
-    if (containerHandlers.getApiBag() == null) {
+    if (apiObject == null || containerHandlers.getApiBag() == null) {
       return;
     }
 
@@ -770,12 +769,17 @@ public class CoreContainer {
     }
     logging = LogWatcher.newRegisteredLogWatcher(cfg.getLogWatcherConfig(), loader);
 
+    ClusterPluginsSource pluginsSource =
+        ClusterPluginsSource.loadClusterPluginsSource(this, loader);
+    containerPluginsRegistry =
+        new ContainerPluginsRegistry(this, containerHandlers.getApiBag(), pluginsSource);
+
     ClusterEventProducerFactory clusterEventProducerFactory = new ClusterEventProducerFactory(this);
     clusterEventProducer = clusterEventProducerFactory;
 
     placementPluginFactory =
         new DelegatingPlacementPluginFactory(
-            PlacementPluginFactoryLoader.getDefaultPlacementPluginFactory(cfg, loader));
+            PlacementPluginFactoryLoader.getDefaultPlacementPluginFactory());
 
     containerPluginsRegistry.registerListener(clusterSingletons.getPluginRegistryListener());
     containerPluginsRegistry.registerListener(
@@ -1085,9 +1089,8 @@ public class CoreContainer {
     if (isZooKeeperAware()) {
       containerPluginsRegistry.refresh();
       getZkController().zkStateReader.registerClusterPropertiesListener(containerPluginsRegistry);
-      ContainerPluginsApi containerPluginsApi = new ContainerPluginsApi(this);
-      registerV2ApiIfEnabled(containerPluginsApi.readAPI);
-      registerV2ApiIfEnabled(containerPluginsApi.editAPI);
+      registerV2ApiIfEnabled(pluginsSource.getReadApi());
+      registerV2ApiIfEnabled(pluginsSource.getEditApi());
 
       // initialize the placement plugin factory wrapper
       // with the plugin configuration from the registry
diff --git a/solr/core/src/java/org/apache/solr/core/NodeConfig.java b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
index d970bab5bbd..ef1cbbf2dfd 100644
--- a/solr/core/src/java/org/apache/solr/core/NodeConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
@@ -34,6 +34,9 @@ import java.util.function.Predicate;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.solr.api.ClusterPluginsSource;
+import org.apache.solr.api.ContainerPluginsRegistry;
+import org.apache.solr.api.NodeConfigClusterPluginsSource;
 import org.apache.solr.client.solrj.impl.SolrZkClientTimeout;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -79,7 +82,6 @@ public class NodeConfig {
 
   private final PluginInfo shardHandlerFactoryConfig;
   private final UpdateShardHandlerConfig updateShardHandlerConfig;
-  private final PluginInfo replicaPlacementFactoryConfig;
 
   private final String configSetServiceClass;
 
@@ -117,6 +119,8 @@ public class NodeConfig {
 
   private final PluginInfo tracerConfig;
 
+  private final PluginInfo[] clusterPlugins;
+
   private final String defaultZkHost;
 
   private NodeConfig(
@@ -130,7 +134,6 @@ public class NodeConfig {
       String sharedLibDirectory,
       PluginInfo shardHandlerFactoryConfig,
       UpdateShardHandlerConfig updateShardHandlerConfig,
-      PluginInfo replicaPlacementFactoryConfig,
       String coreAdminHandlerClass,
       Map<String, String> coreAdminHandlerActions,
       String collectionsAdminHandlerClass,
@@ -151,6 +154,7 @@ public class NodeConfig {
       MetricsConfig metricsConfig,
       Map<String, CacheConfig> cachesConfig,
       PluginInfo tracerConfig,
+      PluginInfo[] clusterPlugins,
       String defaultZkHost,
       Set<Path> allowPaths,
       List<String> allowUrls,
@@ -169,7 +173,6 @@ public class NodeConfig {
     this.sharedLibDirectory = sharedLibDirectory;
     this.shardHandlerFactoryConfig = shardHandlerFactoryConfig;
     this.updateShardHandlerConfig = updateShardHandlerConfig;
-    this.replicaPlacementFactoryConfig = replicaPlacementFactoryConfig;
     this.coreAdminHandlerClass = coreAdminHandlerClass;
     this.coreAdminHandlerActions = coreAdminHandlerActions;
     this.collectionsAdminHandlerClass = collectionsAdminHandlerClass;
@@ -190,6 +193,7 @@ public class NodeConfig {
     this.metricsConfig = metricsConfig;
     this.cachesConfig = cachesConfig == null ? Collections.emptyMap() : cachesConfig;
     this.tracerConfig = tracerConfig;
+    this.clusterPlugins = clusterPlugins;
     this.defaultZkHost = defaultZkHost;
     this.allowPaths = allowPaths;
     this.allowUrls = allowUrls;
@@ -211,6 +215,17 @@ public class NodeConfig {
     if (null == this.solrHome) throw new NullPointerException("solrHome");
     if (null == this.loader) throw new NullPointerException("loader");
 
+    if (this.clusterPlugins != null
+        && this.clusterPlugins.length > 0
+        && !ClusterPluginsSource.resolveClassName()
+            .equals(NodeConfigClusterPluginsSource.class.getName())) {
+      throw new SolrException(
+          ErrorCode.SERVER_ERROR,
+          "Cluster plugins found in solr.xml but the property "
+              + ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED
+              + " is set to true. Cluster plugins may only be declared in solr.xml with immutable configs.");
+    }
+
     setupSharedLib();
     initModules();
   }
@@ -308,10 +323,6 @@ public class NodeConfig {
     return updateShardHandlerConfig;
   }
 
-  public PluginInfo getReplicaPlacementFactoryConfig() {
-    return replicaPlacementFactoryConfig;
-  }
-
   public int getCoreLoadThreadCount(boolean zkAware) {
     return coreLoadThreads == null
         ? (zkAware
@@ -417,6 +428,10 @@ public class NodeConfig {
     return tracerConfig;
   }
 
+  public PluginInfo[] getClusterPlugins() {
+    return clusterPlugins;
+  }
+
   /**
    * Extra file paths that will be allowed for core creation, in addition to SOLR_HOME,
    * SOLR_DATA_HOME and coreRootDir
@@ -571,7 +586,6 @@ public class NodeConfig {
     private String hiddenSysProps;
     private PluginInfo shardHandlerFactoryConfig;
     private UpdateShardHandlerConfig updateShardHandlerConfig = UpdateShardHandlerConfig.DEFAULT;
-    private PluginInfo replicaPlacementFactoryConfig;
     private String configSetServiceClass;
     private String coreAdminHandlerClass = DEFAULT_ADMINHANDLERCLASS;
     private Map<String, String> coreAdminHandlerActions = Collections.emptyMap();
@@ -591,6 +605,7 @@ public class NodeConfig {
     private MetricsConfig metricsConfig;
     private Map<String, CacheConfig> cachesConfig;
     private PluginInfo tracerConfig;
+    private PluginInfo[] clusterPlugins;
     private String defaultZkHost;
     private Set<Path> allowPaths = Collections.emptySet();
     private List<String> allowUrls = Collections.emptyList();
@@ -689,12 +704,6 @@ public class NodeConfig {
       return this;
     }
 
-    public NodeConfigBuilder setReplicaPlacementFactoryConfig(
-        PluginInfo replicaPlacementFactoryConfig) {
-      this.replicaPlacementFactoryConfig = replicaPlacementFactoryConfig;
-      return this;
-    }
-
     public NodeConfigBuilder setCoreAdminHandlerClass(String coreAdminHandlerClass) {
       this.coreAdminHandlerClass = coreAdminHandlerClass;
       return this;
@@ -789,6 +798,11 @@ public class NodeConfig {
       return this;
     }
 
+    public NodeConfigBuilder setClusterPlugins(PluginInfo[] clusterPlugins) {
+      this.clusterPlugins = clusterPlugins;
+      return this;
+    }
+
     public NodeConfigBuilder setDefaultZkHost(String defaultZkHost) {
       this.defaultZkHost = defaultZkHost;
       return this;
@@ -880,7 +894,6 @@ public class NodeConfig {
           sharedLibDirectory,
           shardHandlerFactoryConfig,
           updateShardHandlerConfig,
-          replicaPlacementFactoryConfig,
           coreAdminHandlerClass,
           coreAdminHandlerActions,
           collectionsAdminHandlerClass,
@@ -901,6 +914,7 @@ public class NodeConfig {
           metricsConfig,
           cachesConfig,
           tracerConfig,
+          clusterPlugins,
           defaultZkHost,
           allowPaths,
           allowUrls,
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index aeae09b2212..4c15cd9b7a9 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -38,8 +38,11 @@ import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import javax.management.MBeanServer;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
+import org.apache.solr.cloud.ClusterSingleton;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
 import org.apache.solr.common.ConfigNode;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.CollectionUtil;
 import org.apache.solr.common.util.DOMUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
@@ -157,14 +160,13 @@ public class SolrXmlConfig {
     configBuilder.setSolrResourceLoader(loader);
     configBuilder.setUpdateShardHandlerConfig(updateConfig);
     configBuilder.setShardHandlerFactoryConfig(getPluginInfo(root.get("shardHandlerFactory")));
-    configBuilder.setReplicaPlacementFactoryConfig(
-        getPluginInfo(root.get("replicaPlacementFactory")));
     configBuilder.setTracerConfig(getPluginInfo(root.get("tracerConfig")));
     configBuilder.setLogWatcherConfig(loadLogWatcherConfig(root.get("logging")));
     configBuilder.setSolrProperties(loadProperties(root, substituteProperties));
     if (cloudConfig != null) configBuilder.setCloudConfig(cloudConfig);
     configBuilder.setBackupRepositoryPlugins(
         getBackupRepositoryPluginInfos(root.get("backup").getAll("repository")));
+    configBuilder.setClusterPlugins(getClusterPlugins(loader, root));
     // <metrics><hiddenSysProps></metrics> will be removed in Solr 10, but until then, use it if a
     // <hiddenSysProps> is not provided under <solr>.
     // Remove this line in 10.0
@@ -647,6 +649,68 @@ public class SolrXmlConfig {
     return configs;
   }
 
+  private static PluginInfo[] getClusterPlugins(SolrResourceLoader loader, ConfigNode root) {
+    List<PluginInfo> clusterPlugins = new ArrayList<>();
+
+    Collections.addAll(
+        clusterPlugins, getClusterSingletonPluginInfos(loader, root.getAll("clusterSingleton")));
+
+    PluginInfo replicaPlacementFactory = getPluginInfo(root.get("replicaPlacementFactory"));
+    if (replicaPlacementFactory != null) {
+      if (replicaPlacementFactory.name != null
+          && !replicaPlacementFactory.name.equals(PlacementPluginFactory.PLUGIN_NAME)) {
+        throw new SolrException(
+            SolrException.ErrorCode.SERVER_ERROR,
+            "The replicaPlacementFactory name attribute must be "
+                + PlacementPluginFactory.PLUGIN_NAME);
+      }
+      clusterPlugins.add(replicaPlacementFactory);
+    }
+
+    return clusterPlugins.toArray(new PluginInfo[0]);
+  }
+
+  private static PluginInfo[] getClusterSingletonPluginInfos(
+      SolrResourceLoader loader, List<ConfigNode> nodes) {
+    if (nodes == null || nodes.isEmpty()) {
+      return new PluginInfo[0];
+    }
+
+    List<PluginInfo> plugins =
+        nodes.stream()
+            .map(n -> new PluginInfo(n, n.name(), true, true))
+            .collect(Collectors.toList());
+
+    // Cluster plugin names must be unique
+    Set<String> names = CollectionUtil.newHashSet(nodes.size());
+    Set<String> duplicateNames =
+        plugins.stream()
+            .filter(p -> !names.add(p.name))
+            .map(p -> p.name)
+            .collect(Collectors.toSet());
+    if (!duplicateNames.isEmpty()) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Multiple clusterSingleton sections with name '"
+              + String.join("', '", duplicateNames)
+              + "' found in solr.xml");
+    }
+
+    try {
+      plugins.forEach(
+          p -> {
+            loader.findClass(p.className, ClusterSingleton.class);
+          });
+    } catch (ClassCastException e) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "clusterSingleton plugins must implement the interface "
+              + ClusterSingleton.class.getName());
+    }
+
+    return plugins.toArray(new PluginInfo[0]);
+  }
+
   private static MetricsConfig getMetricsConfig(ConfigNode metrics) {
     MetricsConfig.MetricsConfigBuilder builder = new MetricsConfig.MetricsConfigBuilder();
     boolean enabled = metrics.boolAttr("enabled", true);
diff --git a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
index f2cf3dbeb85..9746cb8cdad 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -212,9 +212,10 @@ public class SolrConfigHandler extends RequestHandlerBase
             resp.add(
                 ZNODEVER,
                 Map.of(
-                    ConfigOverlay.NAME, req.getCore().getSolrConfig().getOverlay().getVersion(),
+                    ConfigOverlay.NAME,
+                    req.getCore().getSolrConfig().getOverlay().getVersion(),
                     RequestParams.NAME,
-                        req.getCore().getSolrConfig().getRequestParams().getZnodeVersion()));
+                    req.getCore().getSolrConfig().getRequestParams().getZnodeVersion()));
             boolean isStale = false;
             int expectedVersion = req.getParams().getInt(ConfigOverlay.NAME, -1);
             int actualVersion = req.getCore().getSolrConfig().getOverlay().getVersion();
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 117334b6d4b..910964783f9 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
@@ -22,28 +22,21 @@ import static org.apache.lucene.util.IOUtils.closeWhileHandlingException;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.function.Function;
-import java.util.function.Supplier;
 import org.apache.solr.api.AnnotatedApi;
+import org.apache.solr.api.ClusterPluginsSource;
 import org.apache.solr.api.Command;
 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;
 import org.apache.solr.client.solrj.request.beans.PluginMeta;
-import org.apache.solr.common.cloud.SolrZkClient;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.security.PermissionNameProvider;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,15 +44,18 @@ import org.slf4j.LoggerFactory;
 public class ContainerPluginsApi {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static final String PLUGIN = ZkStateReader.CONTAINER_PLUGINS;
-  private final Supplier<SolrZkClient> zkClientSupplier;
+  public static final String PLUGIN = "plugin";
+
   private final CoreContainer coreContainer;
+  private final ClusterPluginsSource pluginsSource;
+
   public final Read readAPI = new Read();
   public final Edit editAPI = new Edit();
 
-  public ContainerPluginsApi(CoreContainer coreContainer) {
-    this.zkClientSupplier = coreContainer.zkClientSupplier;
+  public ContainerPluginsApi(
+      CoreContainer coreContainer, ClusterPluginsSource clusterPluginsSource) {
     this.coreContainer = coreContainer;
+    this.pluginsSource = clusterPluginsSource;
   }
 
   /** API for reading the current plugin configurations. */
@@ -69,7 +65,7 @@ public class ContainerPluginsApi {
         path = "/cluster/plugin",
         permission = PermissionNameProvider.Name.COLL_READ_PERM)
     public void list(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
-      rsp.add(PLUGIN, plugins(zkClientSupplier));
+      rsp.add(PLUGIN, pluginsSource.plugins());
     }
   }
 
@@ -85,7 +81,7 @@ public class ContainerPluginsApi {
       PluginMeta info = payload.get();
       validateConfig(payload, info);
       if (payload.hasError()) return;
-      persistPlugins(
+      pluginsSource.persistPlugins(
           map -> {
             if (map.containsKey(info.name)) {
               payload.addError(info.name + " already exists");
@@ -98,7 +94,7 @@ public class ContainerPluginsApi {
 
     @Command(name = "remove")
     public void remove(PayloadObj<String> payload) throws IOException {
-      persistPlugins(
+      pluginsSource.persistPlugins(
           map -> {
             if (map.remove(payload.get()) == null) {
               payload.addError("No such plugin: " + payload.get());
@@ -114,7 +110,7 @@ public class ContainerPluginsApi {
       PluginMeta info = payload.get();
       validateConfig(payload, info);
       if (payload.hasError()) return;
-      persistPlugins(
+      pluginsSource.persistPlugins(
           map -> {
             Map<String, Object> existing = (Map<String, Object>) map.get(info.name);
             if (existing == null) {
@@ -158,56 +154,4 @@ public class ContainerPluginsApi {
       closeWhileHandlingException(api);
     }
   }
-
-  /**
-   * Retrieve the current plugin configurations.
-   *
-   * @param zkClientSupplier supplier of {@link SolrZkClient}
-   * @return current plugin configurations, where keys are plugin names and values are {@link
-   *     PluginMeta} plugin metadata.
-   * @throws IOException on IO errors
-   */
-  @SuppressWarnings("unchecked")
-  public static Map<String, Object> plugins(Supplier<SolrZkClient> zkClientSupplier)
-      throws IOException {
-    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(PLUGIN, o -> new LinkedHashMap<>());
-    } catch (KeeperException.NoNodeException e) {
-      return new LinkedHashMap<>();
-    } catch (KeeperException | InterruptedException e) {
-      throw new IOException("Error reading cluster property", SolrZkClient.checkInterrupted(e));
-    }
-  }
-
-  private void persistPlugins(Function<Map<String, Object>, Map<String, Object>> modifier)
-      throws IOException {
-    try {
-      zkClientSupplier
-          .get()
-          .atomicUpdate(
-              ZkStateReader.CLUSTER_PROPS,
-              bytes -> {
-                @SuppressWarnings("unchecked")
-                Map<String, Object> rawJson =
-                    bytes == null
-                        ? new LinkedHashMap<>()
-                        : (Map<String, Object>) Utils.fromJSON(bytes);
-                @SuppressWarnings("unchecked")
-                Map<String, Object> pluginsModified =
-                    modifier.apply(
-                        (Map<String, Object>)
-                            rawJson.computeIfAbsent(PLUGIN, o -> new LinkedHashMap<>()));
-                if (pluginsModified == null) return null;
-                rawJson.put(PLUGIN, pluginsModified);
-                return Utils.toJSON(rawJson);
-              });
-    } catch (KeeperException | InterruptedException e) {
-      throw new IOException("Error reading cluster property", SolrZkClient.checkInterrupted(e));
-    }
-  }
 }
diff --git a/solr/core/src/test-files/solr/solr-50-all.xml b/solr/core/src/test-files/solr/solr-50-all.xml
index 0c212d72bd7..efa2abd4e6a 100644
--- a/solr/core/src/test-files/solr/solr-50-all.xml
+++ b/solr/core/src/test-files/solr/solr-50-all.xml
@@ -76,4 +76,7 @@
     <repository name="local" class="a.b.C" default="true"/>
   </backup>
 
+  <clusterSingleton name="testSingleton1" class="org.apache.solr.core.TestSolrXml$CS"/>
+  <clusterSingleton name="testSingleton2" class="org.apache.solr.core.TestSolrXml$CS"/>
+
 </solr>
diff --git a/solr/core/src/test/org/apache/solr/api/NodeConfigClusterPluginsSourceTest.java b/solr/core/src/test/org/apache/solr/api/NodeConfigClusterPluginsSourceTest.java
new file mode 100644
index 00000000000..158a777f241
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/api/NodeConfigClusterPluginsSourceTest.java
@@ -0,0 +1,219 @@
+/*
+ * 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.api;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
+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.ClusterSingleton;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.annotation.JsonProperty;
+import org.apache.solr.common.util.ReflectMapWriter;
+import org.apache.solr.core.CoreContainer;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/** Tests that verify initialization of cluster plugins that are declared in solr.xml */
+public class NodeConfigClusterPluginsSourceTest extends SolrCloudTestCase {
+
+  // Any random value for the config parameter
+  private static int CFG_VAL;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    System.setProperty(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, "false");
+    CFG_VAL = random().nextInt();
+    configureCluster(1)
+        .withSolrXml(
+            MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML.replace(
+                "</solr>",
+                SingletonNoConfig.configXml()
+                    + SingletonWithConfig.configXml(new SingletonConfig(CFG_VAL))
+                    + "</solr>"))
+        .addConfig(
+            "conf", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+        .configure();
+  }
+
+  /**
+   * Verifies that the cluster singleton configs declared in solr.xml are loaded into the registry
+   */
+  public void testClusterSingletonsRegistered() {
+
+    CoreContainer cc = cluster.getJettySolrRunners().get(0).getCoreContainer();
+    assertEquals(
+        "expected 2 plugins to be installed to the container plugins registry",
+        2,
+        cc.getNodeConfig().getClusterPlugins().length);
+
+    ContainerPluginsRegistry registry = cc.getContainerPluginsRegistry();
+    registry.refresh();
+
+    // Verify config for SingletonNoConfig
+    ContainerPluginsRegistry.ApiInfo apiInfo = registry.getPlugin(SingletonNoConfig.NAME);
+    assertNotNull("no plugin found in registry for " + SingletonNoConfig.NAME, apiInfo);
+    assertEquals("incorrect plugin name", SingletonNoConfig.NAME, apiInfo.getInfo().name);
+    assertEquals(
+        "incorrect plugin class name", SingletonNoConfig.class.getName(), apiInfo.getInfo().klass);
+    assertNull(
+        "config should not be set because none was specified in solr.xml",
+        apiInfo.getInfo().config);
+
+    // Verify config for SingletonWithConfig
+    apiInfo = registry.getPlugin(SingletonWithConfig.NAME);
+    assertNotNull("no plugin found in registry for " + SingletonWithConfig.NAME, apiInfo);
+    assertEquals("incorrect plugin name", SingletonWithConfig.NAME, apiInfo.getInfo().name);
+    assertEquals(
+        "incorrect plugin class name",
+        SingletonWithConfig.class.getName(),
+        apiInfo.getInfo().klass);
+    MapWriter config = apiInfo.getInfo().config;
+    assertNotNull("config should be set for " + SingletonWithConfig.NAME, config);
+    Map<String, Object> configMap = new HashMap<>();
+    config.toMap(configMap);
+    assertEquals("incorrect config val for cfgInt parameter", CFG_VAL, configMap.get("cfgInt"));
+  }
+
+  /**
+   * Verify that the container plugins Read Api is available and works with plugin configs declared
+   * in solr.xml
+   */
+  @Test
+  public void testClusterPluginsReadApi() throws Exception {
+    V2Response rsp =
+        new V2Request.Builder("/cluster/plugin").GET().build().process(cluster.getSolrClient());
+    assertEquals(0, rsp.getStatus());
+    assertEquals(
+        SingletonNoConfig.class.getName(),
+        rsp._getStr("/plugin/" + SingletonNoConfig.NAME + "/class", null));
+
+    assertEquals(
+        SingletonWithConfig.class.getName(),
+        rsp._getStr("/plugin/" + SingletonWithConfig.NAME + "/class", null));
+  }
+
+  /** Verify that the Edit Apis are not available for plugins declared in solr.xml */
+  @Test
+  public void testClusterPluginsEditApi() throws Exception {
+    PluginMeta meta = SingletonNoConfig.pluginMeta();
+    V2Request req =
+        new V2Request.Builder("/cluster/plugin")
+            .POST()
+            .withPayload(Collections.singletonMap("add", meta))
+            .build();
+    try {
+      req.process(cluster.getSolrClient());
+      fail("Expected a 404 response code because the Edit Apis are not registered");
+    } catch (BaseHttpSolrClient.RemoteExecutionException e) {
+      assertEquals(
+          "Expected a HTTP 404 response code because the /cluster/plugin API should not be registered",
+          404,
+          e.code());
+    }
+  }
+
+  public static class SingletonNoConfig implements ClusterSingleton {
+
+    static final String NAME = "singleton-no-config";
+
+    static String configXml() {
+      return "<clusterSingleton name=\""
+          + NAME
+          + "\" class=\""
+          + SingletonNoConfig.class.getName()
+          + "\"/>";
+    }
+
+    static PluginMeta pluginMeta() {
+      PluginMeta plugin = new PluginMeta();
+      plugin.name = SingletonNoConfig.NAME;
+      plugin.klass = SingletonNoConfig.class.getName();
+      return plugin;
+    }
+
+    @Override
+    public String getName() {
+      return NAME;
+    }
+
+    @Override
+    public void start() throws Exception {}
+
+    @Override
+    public State getState() {
+      return State.STOPPED;
+    }
+
+    @Override
+    public void stop() {}
+  }
+
+  public static class SingletonConfig implements ReflectMapWriter {
+
+    @JsonProperty public int cfgInt;
+
+    public SingletonConfig() {
+      this(-1);
+    }
+
+    public SingletonConfig(int cfgInt) {
+      this.cfgInt = cfgInt;
+    }
+  }
+
+  public static class SingletonWithConfig
+      implements ClusterSingleton, ConfigurablePlugin<SingletonConfig> {
+
+    static final String NAME = "singleton-with-config";
+
+    static String configXml(SingletonConfig config) {
+      return "<clusterSingleton name=\""
+          + NAME
+          + "\" class=\""
+          + SingletonWithConfig.class.getName()
+          + "\"><int name=\"cfgInt\">"
+          + config.cfgInt
+          + "</int></clusterSingleton>";
+    }
+
+    @Override
+    public void configure(SingletonConfig cfg) {}
+
+    @Override
+    public String getName() {
+      return NAME;
+    }
+
+    @Override
+    public void start() throws Exception {}
+
+    @Override
+    public State getState() {
+      return State.STOPPED;
+    }
+
+    @Override
+    public void stop() {}
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/NodeConfigPlacementPluginTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/NodeConfigPlacementPluginTest.java
new file mode 100644
index 00000000000..8608c14c887
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/NodeConfigPlacementPluginTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.placement.impl;
+
+import static org.hamcrest.Matchers.instanceOf;
+
+import org.apache.solr.api.ContainerPluginsRegistry;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.cluster.placement.plugins.AffinityPlacementConfig;
+import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
+import org.apache.solr.core.CoreContainer;
+import org.hamcrest.MatcherAssert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class NodeConfigPlacementPluginTest extends SolrCloudTestCase {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    System.setProperty(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, "false");
+    String pluginXml =
+        "<replicaPlacementFactory class=\"org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory\"><int name=\"minimalFreeDiskGB\">10</int><int name=\"prioritizedFreeDiskGB\">200</int></replicaPlacementFactory>";
+
+    configureCluster(1)
+        .withSolrXml(
+            MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML.replace("</solr>", pluginXml) + "</solr>")
+        .addConfig(
+            "conf", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+        .configure();
+  }
+
+  @Test
+  public void testConfigurationInSolrXml() {
+    CoreContainer cc = cluster.getJettySolrRunner(0).getCoreContainer();
+    MatcherAssert.assertThat(
+        cc.getPlacementPluginFactory().createPluginInstance(),
+        instanceOf(AffinityPlacementFactory.AffinityPlacementPlugin.class));
+    MatcherAssert.assertThat(
+        cc.getPlacementPluginFactory().getConfig(), instanceOf(AffinityPlacementConfig.class));
+
+    AffinityPlacementConfig config =
+        (AffinityPlacementConfig) cc.getPlacementPluginFactory().getConfig();
+    assertEquals(config.minimalFreeDiskGB, 10);
+    assertEquals(config.prioritizedFreeDiskGB, 200);
+    cc.shutdown();
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
index fadc23c6b27..cb1cd6600c0 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
@@ -124,25 +124,6 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
     cc.shutdown();
   }
 
-  @Test
-  public void testConfigurationInSolrXml() {
-    String solrXml =
-        "<solr><replicaPlacementFactory class=\"org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory\"><int name=\"minimalFreeDiskGB\">10</int><int name=\"prioritizedFreeDiskGB\">200</int></replicaPlacementFactory></solr>";
-    CoreContainer cc = createCoreContainer(TEST_PATH(), solrXml);
-
-    MatcherAssert.assertThat(
-        cc.getPlacementPluginFactory().createPluginInstance(),
-        instanceOf(AffinityPlacementFactory.AffinityPlacementPlugin.class));
-    MatcherAssert.assertThat(
-        cc.getPlacementPluginFactory().getConfig(), instanceOf(AffinityPlacementConfig.class));
-
-    AffinityPlacementConfig config =
-        (AffinityPlacementConfig) cc.getPlacementPluginFactory().getConfig();
-    assertEquals(config.minimalFreeDiskGB, 10);
-    assertEquals(config.prioritizedFreeDiskGB, 200);
-    cc.shutdown();
-  }
-
   @Test
   public void testMinimizeCores() throws Exception {
     PluginMeta plugin = new PluginMeta();
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrXml.java b/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
index 9140d580d55..5454baceb86 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
@@ -30,6 +30,9 @@ import java.util.stream.Collectors;
 import org.apache.commons.exec.OS;
 import org.apache.lucene.tests.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.api.ContainerPluginsRegistry;
+import org.apache.solr.cloud.ClusterSingleton;
+import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.CacheConfig;
 import org.apache.solr.search.CaffeineCache;
@@ -56,6 +59,7 @@ public class TestSolrXml extends SolrTestCaseJ4 {
 
     System.setProperty(
         "solr.allowPaths", OS.isFamilyWindows() ? "C:\\tmp,C:\\home\\john" : "/tmp,/home/john");
+    System.setProperty(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, "false");
     NodeConfig cfg = SolrXmlConfig.fromSolrHome(solrHome, new Properties());
     CloudConfig ccfg = cfg.getCloudConfig();
     UpdateShardHandlerConfig ucfg = cfg.getUpdateShardHandlerConfig();
@@ -140,7 +144,18 @@ public class TestSolrXml extends SolrTestCaseJ4 {
     assertTrue("hideStackTrace", cfg.hideStackTraces());
     System.clearProperty("solr.allowPaths");
 
-    PluginInfo replicaPlacementFactoryConfig = cfg.getReplicaPlacementFactoryConfig();
+    PluginInfo[] clusterPlugins = cfg.getClusterPlugins();
+    assertEquals(3, clusterPlugins.length);
+
+    PluginInfo cs1 = cfg.getClusterPlugins()[0];
+    assertEquals("testSingleton1", cs1.name);
+    assertEquals(CS.class.getName(), cs1.className);
+
+    PluginInfo cs2 = cfg.getClusterPlugins()[1];
+    assertEquals("testSingleton2", cs2.name);
+    assertEquals(CS.class.getName(), cs2.className);
+
+    PluginInfo replicaPlacementFactoryConfig = cfg.getClusterPlugins()[2];
     assertEquals(
         "org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory",
         replicaPlacementFactoryConfig.className);
@@ -151,6 +166,7 @@ public class TestSolrXml extends SolrTestCaseJ4 {
   // Test  a few property substitutions that happen to be in solr-50-all.xml.
   public void testPropertySub() throws IOException {
 
+    System.setProperty(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, "false");
     System.setProperty("coreRootDirectory", "myCoreRoot" + File.separator);
     System.setProperty("hostPort", "8888");
     System.setProperty("shareSchema", "false");
@@ -483,4 +499,112 @@ public class TestSolrXml extends SolrTestCaseJ4 {
         assertThrows(SolrException.class, () -> SolrXmlConfig.fromString(solrHome, solrXml));
     assertEquals("Multiple instances of backup section found in solr.xml", thrown.getMessage());
   }
+
+  public void testFailAtConfigParseTimeWhenClusterSingletonHasNoName() {
+    System.setProperty(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, "false");
+    String solrXml =
+        "<solr><clusterPluginsSource class=\"org.apache.solr.api.NodeConfigClusterPluginsSource\"/><clusterSingleton class=\"k1\"/></solr>";
+    RuntimeException thrown =
+        assertThrows(RuntimeException.class, () -> SolrXmlConfig.fromString(solrHome, solrXml));
+    assertEquals(
+        "java.lang.RuntimeException: clusterSingleton: missing mandatory attribute 'name'",
+        thrown.getMessage());
+  }
+
+  public void testFailAtConfigParseTimeWhenClusterSingletonHasDuplicateName() {
+    System.setProperty(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, "false");
+    String solrXml =
+        "<solr><clusterPluginsSource class=\"org.apache.solr.api.NodeConfigClusterPluginsSource\"/><clusterSingleton name=\"a\" class=\"k1\"/><clusterSingleton name=\"a\" class=\"k2\"/></solr>";
+    SolrException thrown =
+        assertThrows(SolrException.class, () -> SolrXmlConfig.fromString(solrHome, solrXml));
+    assertEquals(
+        "Multiple clusterSingleton sections with name 'a' found in solr.xml", thrown.getMessage());
+  }
+
+  public void testFailAtConfigParseTimeWhenClusterSingletonHasNoClass() {
+    System.setProperty(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, "false");
+    String solrXml =
+        "<solr><clusterPluginsSource class=\"org.apache.solr.api.NodeConfigClusterPluginsSource\"/><clusterSingleton name=\"a\"/></solr>";
+    RuntimeException thrown =
+        assertThrows(RuntimeException.class, () -> SolrXmlConfig.fromString(solrHome, solrXml));
+    assertEquals(
+        "java.lang.RuntimeException: clusterSingleton: missing mandatory attribute 'class'",
+        thrown.getMessage());
+  }
+
+  public void testFailAtConfigParseTimeWhenClusterSingletonWithWrongPluginsSource() {
+    String solrXml =
+        "<solr><clusterSingleton name=\"a\" class=\"" + CS.class.getName() + "\"/></solr>";
+    SolrException thrown =
+        assertThrows(SolrException.class, () -> SolrXmlConfig.fromString(solrHome, solrXml));
+    assertEquals(
+        "Cluster plugins found in solr.xml but the property solr.cluster.plugin.edit.enabled is set to true. Cluster plugins may only be declared in solr.xml with immutable configs.",
+        thrown.getMessage());
+  }
+
+  public void testFailAtConfigParseTimeWhenClusterSingletonClassNotFound() {
+    System.setProperty(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, "false");
+    String solrXml = "<solr><clusterSingleton name=\"a\" class=\"class.not.found.Class\"/></solr>";
+    SolrException thrown =
+        assertThrows(SolrException.class, () -> SolrXmlConfig.fromString(solrHome, solrXml));
+    assertEquals(" Error loading class 'class.not.found.Class'", thrown.getMessage());
+  }
+
+  public void testFailAtConfigParseTimeWhenClusterSingletonClassHierarchyIllegal() {
+    System.setProperty(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, "false");
+    String solrXml =
+        "<solr><clusterSingleton name=\"a\" class=\"" + NotCS.class.getName() + "\"/></solr>";
+    SolrException thrown =
+        assertThrows(SolrException.class, () -> SolrXmlConfig.fromString(solrHome, solrXml));
+    assertEquals(
+        "clusterSingleton plugins must implement the interface " + ClusterSingleton.class.getName(),
+        thrown.getMessage());
+  }
+
+  /**
+   * It is not necessary to set the name attribute, but if it is set, ".placement-plugin" is
+   * acceptable
+   */
+  public void testReplicaPlacementFactoryNameCanBeSet() {
+    System.setProperty(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, "false");
+    String solrXml =
+        "<solr><replicaPlacementFactory name=\".placement-plugin\" class=\""
+            + AffinityPlacementFactory.class.getName()
+            + "\"/></solr>";
+    SolrXmlConfig.fromString(solrHome, solrXml);
+  }
+
+  public void testFailAtConfigParseTimeWhenReplicaPlacementFactoryNameIsInvalid() {
+    System.setProperty(ContainerPluginsRegistry.CLUSTER_PLUGIN_EDIT_ENABLED, "false");
+    String solrXml =
+        "<solr><replicaPlacementFactory name=\".must-be-placement-plugin\" class=\""
+            + AffinityPlacementFactory.class.getName()
+            + "\"/></solr>";
+    SolrException thrown =
+        assertThrows(SolrException.class, () -> SolrXmlConfig.fromString(solrHome, solrXml));
+    assertEquals(
+        "The replicaPlacementFactory name attribute must be .placement-plugin",
+        thrown.getMessage());
+  }
+
+  public static class CS implements ClusterSingleton {
+
+    @Override
+    public String getName() {
+      return null;
+    }
+
+    @Override
+    public void start() throws Exception {}
+
+    @Override
+    public State getState() {
+      return null;
+    }
+
+    @Override
+    public void stop() {}
+  }
+
+  public static class NotCS {}
 }
diff --git a/solr/solr-ref-guide/modules/configuration-guide/pages/cluster-plugins.adoc b/solr/solr-ref-guide/modules/configuration-guide/pages/cluster-plugins.adoc
index d4cae1bb40e..bdc6cd160ef 100644
--- a/solr/solr-ref-guide/modules/configuration-guide/pages/cluster-plugins.adoc
+++ b/solr/solr-ref-guide/modules/configuration-guide/pages/cluster-plugins.adoc
@@ -23,7 +23,7 @@ Cluster plugins are pluggable components that are defined and instantiated at th
 These components usually provide admin-level functionality and APIs for additional functionality at the Solr node level.
 
 === Plugin Configurations
-Plugin configurations are maintained using `/cluster/plugin` API.
+If the `solr.cluster.plugin.edit.enabled` property is set to true (the default), then plugin configurations can be maintained using `/cluster/plugin` API.
 
 This API endpoint allows adding, removing and updating plugin configurations.
 
@@ -156,6 +156,8 @@ However, their start/stop life-cycle, as defined in the interface, is controlled
 Any plugin can implement this interface to indicate to Solr that
 it requires this cluster singleton behavior.
 
+`ClusterSingleton` plugins can also be configured by declaring them in xref:configuring-solr-xml.adoc[solr.xml].
+
 === ClusterEventProducer Plugins
 
 In order to support the generation of cluster-level events an implementation of
diff --git a/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc b/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc
index b96413793de..ed169b989e7 100644
--- a/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc
+++ b/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc
@@ -658,6 +658,7 @@ If only `dividend` routing is desired, `hash` may be explicitly set to the empty
 === The <replicaPlacementFactory> Element
 
 A default xref:replica-placement-plugins.adoc[replica placement plugin] can be defined in `solr.xml`.
+To allow this, the `solr.cluster.plugin.edit.enabled` System Property must be set to false. This setting will disable the `/cluster/plugins` edit APIs, preventing modification of cluster plugins at runtime.
 
 [source,xml]
 ----
@@ -670,6 +671,24 @@ A default xref:replica-placement-plugins.adoc[replica placement plugin] can be d
 The `class` attribute should be set to the FQN (fully qualified name) of a class that extends `PlacementPluginFactory`.
 Sub-elements are specific to the implementation.
 
+=== The <clusterSingleton> Element
+One or more `clusterSingleton` elements may be declared in solr.xml.
+To allow this, the `solr.cluster.plugin.edit.enabled` System Property must be set to false. This setting will disable the `/cluster/plugins` edit APIs, preventing modification of cluster plugins at runtime.
+
+Each `clusterSingleton` element specifies a cluster plugin that should be loaded when Solr stars, along with it's associated configuration.
+
+[source,xml]
+----
+<clusterSingleton name="pluginName" class="qualified.plugin.class">
+  <int name="value1">20</int>
+</clusterSingleton>
+----
+
+The `name` attribute is required and must be unique for each `clusterSingleton`.
+
+The `class` attribute should be set to the FQN (fully qualified name) of a class that extends `ClusterSingleton`.
+Sub-elements are specific to the implementation, `value1` is provided as an example here.
+
 === The <metrics> Element
 
 The `<metrics>` element in `solr.xml` allows you to customize the metrics reported by Solr.