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

[lucene] 21/33: SOLR-15016: Initial changes. These also include changes in PR 2099.

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

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

commit 4d08c71c922658a1056fe7c4681cfd276de79e15
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Fri Nov 27 10:35:58 2020 +0100

    SOLR-15016: Initial changes. These also include changes in PR 2099.
---
 .../apache/solr/api/ContainerPluginsRegistry.java  |   4 +-
 .../solr/cloud/api/collections/AddReplicaCmd.java  |  16 +-
 .../apache/solr/cloud/api/collections/Assign.java  |  14 +-
 .../cloud/api/collections/CreateCollectionCmd.java |  14 +-
 .../solr/cloud/api/collections/ReplaceNodeCmd.java |   4 +-
 .../solr/cloud/api/collections/RestoreCmd.java     |   4 +-
 .../solr/cloud/api/collections/SplitShardCmd.java  |   4 +-
 .../events/impl/ClusterEventProducerFactory.java   |   2 +-
 .../impl/CollectionsRepairEventListener.java       |   5 +-
 .../solr/cluster/placement/PlacementPlugin.java    |   5 +
 .../cluster/placement/PlacementPluginConfig.java   |   9 -
 .../cluster/placement/PlacementPluginFactory.java  |   7 +-
 .../placement/impl/PlacementPlanFactoryImpl.java   |   3 +
 .../placement/impl/PlacementPluginConfigImpl.java  | 198 ---------------------
 .../impl/PlacementPluginFactoryLoader.java         |  72 ++++++++
 .../placement/plugins/AffinityPlacementConfig.java |  31 ++++
 .../plugins/AffinityPlacementFactory.java          |  17 +-
 .../plugins/MinimizeCoresPlacementFactory.java     |   2 +-
 .../placement/plugins/RandomPlacementFactory.java  |   2 +-
 .../java/org/apache/solr/core/CoreContainer.java   |  11 ++
 .../java/org/apache/solr/handler/ClusterAPI.java   |   9 +-
 .../impl/PlacementPluginIntegrationTest.java       |   4 +-
 .../plugins/AffinityPlacementFactoryTest.java      |   8 +-
 .../apache/solr/handler/TestContainerPlugin.java   |  20 +--
 .../client/solrj/request/beans/PluginMeta.java     |  12 +-
 25 files changed, 209 insertions(+), 268 deletions(-)

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 158bcf6..9b5c6a1 100644
--- a/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
+++ b/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
@@ -392,8 +392,8 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
       }
       if (instance instanceof ConfigurablePlugin) {
         Class<? extends MapWriter> c = getConfigClass((ConfigurablePlugin<? extends MapWriter>) instance);
-        if (c != null) {
-          MapWriter initVal = mapper.readValue(Utils.toJSON(holder.original), c);
+        if (c != null && holder.meta.config != null) {
+          MapWriter initVal = mapper.readValue(Utils.toJSON(holder.meta.config), c);
           ((ConfigurablePlugin) instance).configure(initVal);
         }
       }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
index 9897f07..11c45c0 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
@@ -20,12 +20,7 @@ package org.apache.solr.cloud.api.collections;
 
 import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.CREATE_NODE_SET;
 import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE;
-import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
-import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
-import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.*;
 import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
 import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
@@ -50,6 +45,7 @@ import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.cloud.ActiveReplicaWatcher;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
+import org.apache.solr.cluster.placement.PlacementPlugin;
 import org.apache.solr.common.SolrCloseableLatch;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -144,7 +140,8 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
       }
     }
 
-    List<CreateReplica> createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount)
+    List<CreateReplica> createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount,
+        ocmh.overseer.getCoreContainer().getPlacementPluginFactory().createPluginInstance())
           .stream()
           .map(replicaPosition -> assignReplicaDetails(ocmh.cloudManager, clusterState, message, replicaPosition))
           .collect(Collectors.toList());
@@ -305,7 +302,8 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
 
   public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
                                                             String collectionName, ZkNodeProps message,
-                                                            EnumMap<Replica.Type, Integer> replicaTypeVsCount) throws IOException, InterruptedException {
+                                                            EnumMap<Replica.Type, Integer> replicaTypeVsCount,
+                                                            PlacementPlugin placementPlugin) throws IOException, InterruptedException {
     boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
     boolean skipNodeAssignment = message.getBool(CollectionAdminParams.SKIP_NODE_ASSIGNMENT, false);
     String sliceName = message.getStr(SHARD_ID_PROP);
@@ -329,7 +327,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     if (!skipCreateReplicaInClusterState && !skipNodeAssignment) {
 
       positions = Assign.getNodesForNewReplicas(clusterState, collection.getName(), sliceName, numNrtReplicas,
-                    numTlogReplicas, numPullReplicas, createNodeSetStr, cloudManager);
+                    numTlogReplicas, numPullReplicas, createNodeSetStr, cloudManager, placementPlugin);
     }
 
     if (positions == null)  {
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
index 968fb92..0c249c9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
@@ -42,7 +42,6 @@ import org.apache.solr.client.solrj.cloud.BadVersionException;
 import org.apache.solr.client.solrj.cloud.VersionedData;
 import org.apache.solr.cluster.placement.PlacementPlugin;
 import org.apache.solr.cluster.placement.impl.PlacementPluginAssignStrategy;
-import org.apache.solr.cluster.placement.impl.PlacementPluginConfigImpl;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
@@ -270,7 +269,8 @@ public class Assign {
   @SuppressWarnings({"unchecked"})
   public static List<ReplicaPosition> getNodesForNewReplicas(ClusterState clusterState, String collectionName,
                                                           String shard, int nrtReplicas, int tlogReplicas, int pullReplicas,
-                                                          Object createNodeSet, SolrCloudManager cloudManager) throws IOException, InterruptedException, AssignmentException {
+                                                          Object createNodeSet, SolrCloudManager cloudManager,
+                                                          PlacementPlugin placementPlugin) throws IOException, InterruptedException, AssignmentException {
     log.debug("getNodesForNewReplicas() shard: {} , nrtReplicas : {} , tlogReplicas: {} , pullReplicas: {} , createNodeSet {}"
         , shard, nrtReplicas, tlogReplicas, pullReplicas, createNodeSet);
     DocCollection coll = clusterState.getCollection(collectionName);
@@ -296,7 +296,7 @@ public class Assign {
         .assignPullReplicas(pullReplicas)
         .onNodes(createNodeList)
         .build();
-    AssignStrategy assignStrategy = createAssignStrategy(cloudManager, clusterState, coll);
+    AssignStrategy assignStrategy = createAssignStrategy(placementPlugin, clusterState, coll);
     return assignStrategy.assign(cloudManager, assignRequest);
   }
 
@@ -493,12 +493,10 @@ public class Assign {
    * Creates the appropriate instance of {@link AssignStrategy} based on how the cluster and/or individual collections are
    * configured.
    */
-  public static AssignStrategy createAssignStrategy(SolrCloudManager solrCloudManager, ClusterState clusterState, DocCollection collection) {
-    PlacementPlugin plugin = PlacementPluginConfigImpl.getPlacementPlugin(solrCloudManager);
-
-    if (plugin != null) {
+  public static AssignStrategy createAssignStrategy(PlacementPlugin placementPlugin, ClusterState clusterState, DocCollection collection) {
+    if (placementPlugin != null) {
       // If a cluster wide placement plugin is configured (and that's the only way to define a placement plugin)
-      return new PlacementPluginAssignStrategy(collection, plugin);
+      return new PlacementPluginAssignStrategy(collection, placementPlugin);
     }  else {
         return new LegacyAssignStrategy();
       }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
index c938d75..adc9728 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
@@ -41,6 +41,7 @@ import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
 import org.apache.solr.cloud.overseer.ClusterStateMutator;
+import org.apache.solr.cluster.placement.PlacementPlugin;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.Aliases;
@@ -168,7 +169,8 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
 
       List<ReplicaPosition> replicaPositions = null;
       try {
-        replicaPositions = buildReplicaPositions(ocmh.cloudManager, clusterState, clusterState.getCollection(collectionName), message, shardNames);
+        replicaPositions = buildReplicaPositions(ocmh.cloudManager, clusterState, clusterState.getCollection(collectionName),
+            message, shardNames, ocmh.overseer.getCoreContainer().getPlacementPluginFactory().createPluginInstance());
       } catch (Assign.AssignmentException e) {
         ZkNodeProps deleteMessage = new ZkNodeProps("name", collectionName);
         new DeleteCollectionCmd(ocmh).call(clusterState, deleteMessage, results);
@@ -287,10 +289,10 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
     }
   }
 
-  public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
-                                                            DocCollection docCollection,
-                                                            ZkNodeProps message,
-                                                            List<String> shardNames) throws IOException, InterruptedException, Assign.AssignmentException {
+  private static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
+                                                             DocCollection docCollection,
+                                                             ZkNodeProps message,
+                                                             List<String> shardNames, PlacementPlugin placementPlugin) throws IOException, InterruptedException, Assign.AssignmentException {
     final String collectionName = message.getStr(NAME);
     // look at the replication factor and see if it matches reality
     // if it does not, find best nodes to create more cores
@@ -329,7 +331,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
           .assignPullReplicas(numPullReplicas)
           .onNodes(nodeList)
           .build();
-      Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(cloudManager, clusterState, docCollection);
+      Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(placementPlugin, clusterState, docCollection);
       replicaPositions = assignStrategy.assign(cloudManager, assignRequest);
     }
     return replicaPositions;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
index 2267b4d..271677f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
@@ -120,7 +120,9 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
               .assignPullReplicas(numPullReplicas)
               .onNodes(new ArrayList<>(ocmh.cloudManager.getClusterStateProvider().getLiveNodes()))
               .build();
-          Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ocmh.cloudManager, clusterState, clusterState.getCollection(sourceCollection));
+          Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(
+              ocmh.overseer.getCoreContainer().getPlacementPluginFactory().createPluginInstance(),
+              clusterState, clusterState.getCollection(sourceCollection));
           targetNode = assignStrategy.assign(ocmh.cloudManager, assignRequest).get(0).node;
         }
         ZkNodeProps msg = sourceReplica.plus("parallel", String.valueOf(parallel)).plus(CoreAdminParams.NODE, targetNode);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
index db408b4..c7c941a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
@@ -229,7 +229,9 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
             .assignPullReplicas(numPullReplicas)
             .onNodes(nodeList)
             .build();
-    Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ocmh.cloudManager, clusterState, restoreCollection);
+    Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(
+        ocmh.overseer.getCoreContainer().getPlacementPluginFactory().createPluginInstance(),
+        clusterState, restoreCollection);
     List<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
 
     CountDownLatch countDownLatch = new CountDownLatch(restoreCollection.getSlices().size());
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
index 80aa7d7..012a67e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
@@ -434,7 +434,9 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
           .assignPullReplicas(numPull.get())
           .onNodes(new ArrayList<>(clusterState.getLiveNodes()))
           .build();
-      Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ocmh.cloudManager, clusterState, collection);
+      Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(
+          ocmh.overseer.getCoreContainer().getPlacementPluginFactory().createPluginInstance(),
+          clusterState, collection);
       List<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
       t.stop();
 
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerFactory.java b/solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerFactory.java
index 17f769b..85f1410 100644
--- a/solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerFactory.java
+++ b/solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerFactory.java
@@ -162,8 +162,8 @@ public class ClusterEventProducerFactory extends ClusterEventProducerBase {
 
       @Override
       public void modified(ContainerPluginsRegistry.ApiInfo old, ContainerPluginsRegistry.ApiInfo replacement) {
-        added(replacement);
         deleted(old);
+        added(replacement);
       }
     };
     plugins.registerListener(pluginListener);
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java b/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java
index 48400f8..a1dc136 100644
--- a/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java
+++ b/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java
@@ -41,6 +41,7 @@ import org.apache.solr.cloud.api.collections.Assign;
 import org.apache.solr.cluster.events.ClusterEvent;
 import org.apache.solr.cluster.events.ClusterEventListener;
 import org.apache.solr.cluster.events.NodesDownEvent;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ReplicaPosition;
@@ -76,10 +77,12 @@ public class CollectionsRepairEventListener implements ClusterEventListener, Clu
   private int waitForSecond = DEFAULT_WAIT_FOR_SEC;
 
   private ScheduledThreadPoolExecutor waitForExecutor;
+  private PlacementPluginFactory placementPluginFactory;
 
   public CollectionsRepairEventListener(CoreContainer cc) {
     this.solrClient = cc.getSolrClientCache().getCloudSolrClient(cc.getZkController().getZkClient().getZkServerAddress());
     this.solrCloudManager = cc.getZkController().getSolrCloudManager();
+    this.placementPluginFactory = cc.getPlacementPluginFactory();
   }
 
   @VisibleForTesting
@@ -167,7 +170,7 @@ public class CollectionsRepairEventListener implements ClusterEventListener, Clu
                 .incrementAndGet();
           }
         });
-        Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(solrCloudManager, clusterState, coll);
+        Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(placementPluginFactory.createPluginInstance(), clusterState, coll);
         lostReplicas.forEach((shard, types) -> {
           Assign.AssignRequestBuilder assignRequestBuilder = new Assign.AssignRequestBuilder()
               .forCollection(coll.getName())
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
index bbb52cb..11b08dd 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java
@@ -31,6 +31,11 @@ import org.apache.solr.cluster.Cluster;
  */
 public interface PlacementPlugin {
   /**
+   * Name of the property containing the factory class
+   */
+  String FACTORY_CLASS = "class";
+
+  /**
    * <p>Request from plugin code to compute placement. Note this method must be reentrant as a plugin instance may (read
    * will) get multiple such calls in parallel.
    *
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginConfig.java b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginConfig.java
index d223dcc..1518377 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginConfig.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginConfig.java
@@ -70,15 +70,6 @@ package org.apache.solr.cluster.placement;
 public interface PlacementPluginConfig {
 
   /**
-   * The key in {@code clusterprops.json} under which the plugin factory and the plugin configuration are defined.
-   */
-  String PLACEMENT_PLUGIN_CONFIG_KEY = "placement-plugin";
-  /**
-   * Name of the property containing the factory class
-   */
-  String FACTORY_CLASS = "class";
-
-  /**
    * @return the configured {@link String} value corresponding to {@code configName} if one exists (could be the empty
    * string) and {@code null} otherwise.
    */
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginFactory.java b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginFactory.java
index 7372003..7df4d86 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginFactory.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginFactory.java
@@ -23,9 +23,14 @@ package org.apache.solr.cluster.placement;
  */
 public interface PlacementPluginFactory {
   /**
+   * The key in the plugins registry under which this plugin and its configuration are defined.
+   */
+  String PLUGIN_NAME = "placement-plugin";
+
+  /**
    * Returns an instance of the plugin that will be repeatedly (and concurrently) be called to compute placement. Multiple
    * instances of a plugin can be used in parallel (for example if configuration has to change, but plugin instances with
    * the previous configuration are still being used).
    */
-  PlacementPlugin createPluginInstance(PlacementPluginConfig config);
+  PlacementPlugin createPluginInstance();
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPlanFactoryImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPlanFactoryImpl.java
index 7f7f89f..35671d1 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPlanFactoryImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPlanFactoryImpl.java
@@ -24,6 +24,9 @@ import org.apache.solr.cluster.placement.*;
 
 import java.util.Set;
 
+/**
+ * Simple implementation of {@link PlacementPlanFactory}.
+ */
 public class PlacementPlanFactoryImpl implements PlacementPlanFactory {
   @Override
   public PlacementPlan createPlacementPlan(PlacementRequest request, Set<ReplicaPlacement> replicaPlacements) {
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginConfigImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginConfigImpl.java
deleted file mode 100644
index 30cb6ef..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginConfigImpl.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.cluster.placement.impl;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.cluster.placement.PlacementPlugin;
-import org.apache.solr.cluster.placement.PlacementPluginConfig;
-import org.apache.solr.cluster.placement.PlacementPluginFactory;
-import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.Utils;
-
-/**
- * <p>This concrete class is implementing the config as visible by the placement plugins and contains the code transforming the
- * plugin configuration (currently stored in {@code clusterprops.json} into a strongly typed abstraction (that will not
- * change if internally plugin configuration is moved to some other place).</p>
- *
- * <p>This class also contains the (static) code dealing with instantiating the plugin factory config (it is config, even though
- * of a slightly different type). This code is not accessed by the plugin code but used from the
- * {@link org.apache.solr.cloud.api.collections.Assign} class.</p>
- */
-public class PlacementPluginConfigImpl implements PlacementPluginConfig {
-
-  // Separating configs into typed maps based on the element names in solr.xml
-  private final Map<String, String> stringConfigs;
-  private final Map<String, Long> longConfigs;
-  private final Map<String, Boolean> boolConfigs;
-  private final Map<String, Double> doubleConfigs;
-
-
-  private PlacementPluginConfigImpl(Map<String, String> stringConfigs,
-                                    Map<String, Long> longConfigs,
-                                    Map<String, Boolean> boolConfigs,
-                                    Map<String, Double> doubleConfigs) {
-    this.stringConfigs = stringConfigs;
-    this.longConfigs = longConfigs;
-    this.boolConfigs = boolConfigs;
-    this.doubleConfigs = doubleConfigs;
-  }
-
-  @Override
-  public String getStringConfig(String configName) {
-    return stringConfigs.get(configName);
-  }
-
-  @Override
-  public String getStringConfig(String configName, String defaultValue) {
-    String retval = stringConfigs.get(configName);
-    return retval != null ? retval : defaultValue;
-  }
-
-  @Override
-  public Boolean getBooleanConfig(String configName) {
-    return boolConfigs.get(configName);
-  }
-
-  @Override
-  public Boolean getBooleanConfig(String configName, boolean defaultValue) {
-    Boolean retval = boolConfigs.get(configName);
-    return retval != null ? retval : defaultValue;
-  }
-
-  @Override
-  public Long getLongConfig(String configName) {
-    return longConfigs.get(configName);
-  }
-
-  @Override
-  public Long getLongConfig(String configName, long defaultValue) {
-    Long retval = longConfigs.get(configName);
-    return retval != null ? retval : defaultValue;
-  }
-
-  @Override
-  public Double getDoubleConfig(String configName) {
-    return doubleConfigs.get(configName);
-  }
-
-  @Override
-  public Double getDoubleConfig(String configName, double defaultValue) {
-    Double retval = doubleConfigs.get(configName);
-    return retval != null ? retval : defaultValue;
-  }
-
-  /**
-   * <p>Parses the {@link Map} obtained as the value for key {@link #PLACEMENT_PLUGIN_CONFIG_KEY} from
-   * the {@code clusterprops.json} configuration {@link Map} (obtained by calling
-   * {@link org.apache.solr.client.solrj.impl.ClusterStateProvider#getClusterProperties()}) and translates it into a
-   * configuration consumable by the plugin (and that will not change as Solr changes internally how and where it stores
-   * configuration).</p>
-   *
-   * <p>Configuration properties {@code class} and {@code name} are reserved: for defining the plugin factory class and
-   * a human readable plugin name. All other properties are plugin specific.</p>
-   *
-   * <p>See configuration example and how-to in {@link AffinityPlacementFactory}.</p>
-   */
-  public static PlacementPluginConfig createConfigFromProperties(Map<String, Object> pluginConfig) {
-    final Map<String, String> stringConfigs = new HashMap<>();
-    final Map<String, Long> longConfigs = new HashMap<>();
-    final Map<String, Boolean> boolConfigs = new HashMap<>();
-    final Map<String, Double> doubleConfigs = new HashMap<>();
-
-    for (Map.Entry<String, Object> e : pluginConfig.entrySet()) {
-      String key = e.getKey();
-      if (PlacementPluginConfig.FACTORY_CLASS.equals(key)) {
-        continue;
-      }
-
-      if (key == null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Missing config name attribute in parameter of " + PlacementPluginConfig.PLACEMENT_PLUGIN_CONFIG_KEY);
-      }
-
-      Object value = e.getValue();
-
-      if (value == null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Missing config value for parameter " + key + " of " + PlacementPluginConfig.PLACEMENT_PLUGIN_CONFIG_KEY);
-      }
-
-      if (value instanceof String) {
-        stringConfigs.put(key, (String) value);
-      } else if (value instanceof Long) {
-        longConfigs.put(key, (Long) value);
-      } else if (value instanceof Boolean) {
-        boolConfigs.put(key, (Boolean) value);
-      } else if (value instanceof Double) {
-        doubleConfigs.put(key, (Double) value);
-      } else {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unsupported config type " + value.getClass().getName() +
-            " for parameter " + key + " of " + PlacementPluginConfig.PLACEMENT_PLUGIN_CONFIG_KEY);
-      }
-    }
-
-    return new PlacementPluginConfigImpl(stringConfigs, longConfigs, boolConfigs, doubleConfigs);
-  }
-
-  /**
-   * <p>This is where the plugin configuration is being read (from wherever in Solr it lives, and this will likely change with time),
-   * a {@link org.apache.solr.cluster.placement.PlacementPluginFactory} (as configured) instantiated and a plugin instance
-   * created from this factory.</p>
-   *
-   * <p>The initial implementation you see here is crude! the configuration is read anew each time and the factory class
-   * as well as the plugin class instantiated each time.
-   * This has to be changed once the code is accepted overall, to register a listener that is notified when the configuration
-   * changes (see {@link org.apache.solr.common.cloud.ZkStateReader#registerClusterPropertiesListener})
-   * and that will either create a new instance of the plugin with new configuration using the existing factory (if the factory
-   * class has not changed - we need to keep track of this one) of create a new factory altogether (then a new plugin instance).</p>
-   */
-  @SuppressWarnings({"unchecked"})
-  public static PlacementPlugin getPlacementPlugin(SolrCloudManager solrCloudManager) {
-    Map<String, Object> props = solrCloudManager.getClusterStateProvider().getClusterProperties();
-    Map<String, Object> pluginConfigMap = (Map<String, Object>) props.get(PlacementPluginConfig.PLACEMENT_PLUGIN_CONFIG_KEY);
-
-    if (pluginConfigMap == null) {
-      return null;
-    }
-
-    String pluginFactoryClassName = (String) pluginConfigMap.get(PlacementPluginConfig.FACTORY_CLASS);
-
-    // Get the configured plugin factory class. Is there a way to load a resource in Solr without being in the context of
-    // CoreContainer? Here the placement code is unrelated to the presence of cores (and one can imagine it running on
-    // specialized nodes not having a CoreContainer). I guess the loading code below is not totally satisfying (although
-    // it's not the only place in Solr doing it that way), but I didn't find more satisfying alternatives. Open to suggestions.
-    PlacementPluginFactory placementPluginFactory;
-    try {
-      Class<? extends PlacementPluginFactory> factoryClazz =
-          Class.forName(pluginFactoryClassName, true, PlacementPluginConfigImpl.class.getClassLoader())
-              .asSubclass(PlacementPluginFactory.class);
-
-      placementPluginFactory = factoryClazz.getConstructor().newInstance(); // no args constructor - that's why we introduced a factory...
-    } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to instantiate placement-plugin factory: " +
-          Utils.toJSONString(pluginConfigMap) + " please review /clusterprops.json config for " + PlacementPluginConfig.PLACEMENT_PLUGIN_CONFIG_KEY, e);
-    }
-
-    // Translate the config from the properties where they are defined into the abstraction seen by the plugin
-    PlacementPluginConfig pluginConfig = createConfigFromProperties(pluginConfigMap);
-
-    return placementPluginFactory.createPluginInstance(pluginConfig);
-  }
-}
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
new file mode 100644
index 0000000..f444834
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java
@@ -0,0 +1,72 @@
+package org.apache.solr.cluster.placement.impl;
+
+import org.apache.solr.api.ContainerPluginsRegistry;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+
+/**
+ * Utility class to load the configured {@link PlacementPluginFactory} plugin and
+ * then keep it up to date as the plugin configuration changes.
+ */
+public class PlacementPluginFactoryLoader {
+
+  public static PlacementPluginFactory load(ContainerPluginsRegistry plugins) {
+    final DelegatingPlacementPluginFactory pluginFactory = new DelegatingPlacementPluginFactory();
+    ContainerPluginsRegistry.ApiInfo pluginFactoryInfo = plugins.getPlugin(PlacementPluginFactory.PLUGIN_NAME);
+    if (pluginFactoryInfo != null && (pluginFactoryInfo.getInstance() instanceof PlacementPluginFactory)) {
+      pluginFactory.setDelegate((PlacementPluginFactory) pluginFactoryInfo.getInstance());
+    }
+    ContainerPluginsRegistry.PluginRegistryListener pluginListener = new ContainerPluginsRegistry.PluginRegistryListener() {
+      @Override
+      public void added(ContainerPluginsRegistry.ApiInfo plugin) {
+        if (plugin == null || plugin.getInstance() == null) {
+          return;
+        }
+        Object instance = plugin.getInstance();
+        if (instance instanceof PlacementPluginFactory) {
+          pluginFactory.setDelegate((PlacementPluginFactory) instance);
+        }
+      }
+
+      @Override
+      public void deleted(ContainerPluginsRegistry.ApiInfo plugin) {
+        if (plugin == null || plugin.getInstance() == null) {
+          return;
+        }
+        Object instance = plugin.getInstance();
+        if (instance instanceof PlacementPluginFactory) {
+          pluginFactory.setDelegate(null);
+        }
+      }
+
+      @Override
+      public void modified(ContainerPluginsRegistry.ApiInfo old, ContainerPluginsRegistry.ApiInfo replacement) {
+        deleted(old);
+        added(replacement);
+      }
+    };
+    plugins.registerListener(pluginListener);
+    return pluginFactory;
+  }
+
+  /**
+   * Helper class to support dynamic reloading of plugin implementations.
+   */
+  private static final class DelegatingPlacementPluginFactory implements PlacementPluginFactory {
+
+    private PlacementPluginFactory delegate;
+
+    @Override
+    public PlacementPlugin createPluginInstance() {
+      if (delegate != null) {
+        return delegate.createPluginInstance();
+      } else {
+        return null;
+      }
+    }
+
+    public void setDelegate(PlacementPluginFactory delegate) {
+      this.delegate = delegate;
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementConfig.java b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementConfig.java
new file mode 100644
index 0000000..ae996ef
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementConfig.java
@@ -0,0 +1,31 @@
+package org.apache.solr.cluster.placement.plugins;
+
+import org.apache.solr.common.annotation.JsonProperty;
+import org.apache.solr.common.util.ReflectMapWriter;
+
+import java.io.IOException;
+
+/**
+ *
+ */
+public class AffinityPlacementConfig implements ReflectMapWriter {
+
+  public static final AffinityPlacementConfig DEFAULT = new AffinityPlacementConfig();
+
+  @JsonProperty
+  public long minimalFreeDiskGB;
+
+  @JsonProperty
+  public long prioritizedFreeDiskGB;
+
+  // no-arg public constructor required for deserialization
+  public AffinityPlacementConfig() {
+    minimalFreeDiskGB = 20L;
+    prioritizedFreeDiskGB = 100L;
+  }
+
+  public AffinityPlacementConfig(long minimalFreeDiskGB, long prioritizedFreeDiskGB) {
+    this.minimalFreeDiskGB = minimalFreeDiskGB;
+    this.prioritizedFreeDiskGB = prioritizedFreeDiskGB;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java
index 379e06c..4867ab1 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java
@@ -20,6 +20,7 @@ package org.apache.solr.cluster.placement.plugins;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Ordering;
 import com.google.common.collect.TreeMultimap;
+import org.apache.solr.api.ConfigurablePlugin;
 import org.apache.solr.cluster.*;
 import org.apache.solr.cluster.placement.*;
 import org.apache.solr.common.util.Pair;
@@ -116,7 +117,7 @@ import java.util.stream.Collectors;
  * make it relatively easy to adapt it to (somewhat) different assumptions. Configuration options could be introduced
  * to allow configuration base option selection as well...</p>
  */
-public class AffinityPlacementFactory implements PlacementPluginFactory {
+public class AffinityPlacementFactory implements PlacementPluginFactory, ConfigurablePlugin<AffinityPlacementConfig> {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   /**
@@ -155,6 +156,8 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
    */
   public static final String PRIORITIZED_FREE_DISK_GB = "prioritizedFreeDiskGB";
 
+  private AffinityPlacementConfig config = AffinityPlacementConfig.DEFAULT;
+
   /**
    * Empty public constructor is used to instantiate this factory. Using a factory pattern to allow the factory to do one
    * time costly operations if needed, and to only have to instantiate a default constructor class by name, rather than
@@ -165,10 +168,14 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
   }
 
   @Override
-  public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
-    final long minimalFreeDiskGB = config.getLongConfig(MINIMAL_FREE_DISK_GB, 20L);
-    final long prioritizedFreeDiskGB = config.getLongConfig(PRIORITIZED_FREE_DISK_GB, 100L);
-    return new AffinityPlacementPlugin(minimalFreeDiskGB, prioritizedFreeDiskGB);
+  public PlacementPlugin createPluginInstance() {
+    return new AffinityPlacementPlugin(config.minimalFreeDiskGB, config.prioritizedFreeDiskGB);
+  }
+
+  @Override
+  public void configure(AffinityPlacementConfig cfg) {
+    Objects.requireNonNull(cfg, "configuration must never be null");
+    this.config = cfg;
   }
 
   /**
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/MinimizeCoresPlacementFactory.java b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/MinimizeCoresPlacementFactory.java
index b73b692..d6f88f3 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/MinimizeCoresPlacementFactory.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/MinimizeCoresPlacementFactory.java
@@ -43,7 +43,7 @@ import org.apache.solr.common.util.SuppressForbidden;
 public class MinimizeCoresPlacementFactory implements PlacementPluginFactory {
 
   @Override
-  public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
+  public PlacementPlugin createPluginInstance() {
     return new MinimizeCoresPlacementPlugin();
   }
 
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/RandomPlacementFactory.java b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/RandomPlacementFactory.java
index 27699d8..cec1d9b 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/RandomPlacementFactory.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/RandomPlacementFactory.java
@@ -39,7 +39,7 @@ import org.apache.solr.cluster.placement.*;
 public class RandomPlacementFactory implements PlacementPluginFactory {
 
   @Override
-  public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
+  public PlacementPlugin createPluginInstance() {
     return new RandomPlacementPlugin();
   }
 
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 7331cef..b127263 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -74,6 +74,9 @@ import org.apache.solr.cloud.OverseerTaskQueue;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cluster.events.ClusterEventProducer;
 import org.apache.solr.cluster.events.impl.ClusterEventProducerFactory;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+import org.apache.solr.cluster.placement.impl.PlacementPluginFactoryLoader;
 import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -257,6 +260,7 @@ public class CoreContainer {
 
   // initially these are the same to collect the plugin-based listeners during init
   private ClusterEventProducer clusterEventProducer;
+  private PlacementPluginFactory placementPluginFactory;
 
   private PackageStoreAPI packageStoreAPI;
   private PackageLoader packageLoader;
@@ -896,6 +900,9 @@ public class CoreContainer {
       containerHandlers.getApiBag().registerObject(containerPluginsApi.readAPI);
       containerHandlers.getApiBag().registerObject(containerPluginsApi.editAPI);
 
+      // get the placement plugin
+      placementPluginFactory = PlacementPluginFactoryLoader.load(containerPluginsRegistry);
+
       // create target ClusterEventProducer (possibly from plugins)
       clusterEventProducer = clusterEventProducerFactory.create(containerPluginsRegistry);
 
@@ -2180,6 +2187,10 @@ public class CoreContainer {
     return clusterEventProducer;
   }
 
+  public PlacementPluginFactory getPlacementPluginFactory() {
+    return placementPluginFactory;
+  }
+
   static {
     ExecutorUtil.addThreadLocalProvider(SolrRequestInfo.getInheritableThreadLocalProvider());
   }
diff --git a/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java b/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java
index 605dbb6..157c0e5 100644
--- a/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java
@@ -27,7 +27,8 @@ import org.apache.solr.client.solrj.request.beans.ClusterPropInfo;
 import org.apache.solr.client.solrj.request.beans.CreateConfigInfo;
 import org.apache.solr.client.solrj.request.beans.RateLimiterMeta;
 import org.apache.solr.cloud.OverseerConfigSetMessageHandler;
-import org.apache.solr.cluster.placement.PlacementPluginConfig;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
 import org.apache.solr.common.MapWriterMap;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.annotation.JsonProperty;
@@ -250,14 +251,14 @@ public class ClusterAPI {
       ClusterProperties clusterProperties = new ClusterProperties(getCoreContainer().getZkController().getZkClient());
       // When the json contains { "set-placement-plugin" : null }, the map is empty, not null.
       // Very basic sanity check. Real validation will be done when the config is used...
-      if (!(placementPluginConfig == null) && !placementPluginConfig.containsKey(PlacementPluginConfig.FACTORY_CLASS)) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Must contain " + PlacementPluginConfig.FACTORY_CLASS + " attribute (or be null)");
+      if (!(placementPluginConfig == null) && !placementPluginConfig.containsKey(PlacementPlugin.FACTORY_CLASS)) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Must contain " + PlacementPlugin.FACTORY_CLASS + " attribute (or be null)");
       }
       try {
         clusterProperties.update(placementPluginConfig == null?
             null:
             new MapWriterMap(placementPluginConfig),
-            PlacementPluginConfig.PLACEMENT_PLUGIN_CONFIG_KEY);
+            PlacementPluginFactory.PLUGIN_NAME);
       } catch (Exception e) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error in API", e);
       }
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 2acad7e..90e3965 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
@@ -5,7 +5,7 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.cloud.SolrCloudTestCase;
-import org.apache.solr.cluster.placement.PlacementPluginConfig;
+import org.apache.solr.cluster.placement.PlacementPlugin;
 import org.apache.solr.cluster.placement.plugins.MinimizeCoresPlacementFactory;
 import org.apache.solr.common.cloud.ClusterProperties;
 import org.apache.solr.common.cloud.ClusterState;
@@ -55,7 +55,7 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
 
   @Test
   public void testMinimizeCores() throws Exception {
-    Map<String, Object> config = Map.of(PlacementPluginConfig.FACTORY_CLASS, MinimizeCoresPlacementFactory.class.getName());
+    Map<String, Object> config = Map.of(PlacementPlugin.FACTORY_CLASS, MinimizeCoresPlacementFactory.class.getName());
     V2Request req = new V2Request.Builder("/cluster")
         .forceV2(true)
         .POST()
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactoryTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactoryTest.java
index 3d3c1dc..61907be 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactoryTest.java
@@ -26,7 +26,6 @@ import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.*;
 import org.apache.solr.cluster.placement.Builders;
 import org.apache.solr.cluster.placement.impl.PlacementPlanFactoryImpl;
-import org.apache.solr.cluster.placement.impl.PlacementPluginConfigImpl;
 import org.apache.solr.cluster.placement.impl.PlacementRequestImpl;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -50,9 +49,10 @@ public class AffinityPlacementFactoryTest extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void setupPlugin() {
-    PlacementPluginConfig config = PlacementPluginConfigImpl.createConfigFromProperties(
-        Map.of("minimalFreeDiskGB", 10L, "prioritizedFreeDiskGB", 50L));
-    plugin = new AffinityPlacementFactory().createPluginInstance(config);
+    AffinityPlacementConfig config = new AffinityPlacementConfig(10L, 50L);
+    AffinityPlacementFactory factory = new AffinityPlacementFactory();
+    factory.configure(config);
+    plugin = factory.createPluginInstance();
     ((AffinityPlacementFactory.AffinityPlacementPlugin) plugin).setRandom(random());
   }
 
diff --git a/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java b/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
index 01ab39f..224caf7 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
@@ -193,12 +193,14 @@ public class TestContainerPlugin extends SolrCloudTestCase {
       assertEquals( CConfig.class, ContainerPluginsRegistry.getConfigClass(new CC1()));
       assertEquals( CConfig.class, ContainerPluginsRegistry.getConfigClass(new CC2()));
 
-      CConfig p = new CConfig();
-      p.boolVal = Boolean.TRUE;
-      p.strVal = "Something";
-      p.longVal = 1234L;
+      CConfig cfg = new CConfig();
+      cfg.boolVal = Boolean.TRUE;
+      cfg.strVal = "Something";
+      cfg.longVal = 1234L;
+      PluginMeta p = new PluginMeta();
       p.name = "hello";
       p.klass = CC.class.getName();
+      p.config = cfg;
 
       new V2Request.Builder("/cluster/plugin")
           .forceV2(true)
@@ -213,7 +215,7 @@ public class TestContainerPlugin extends SolrCloudTestCase {
               .build().process(cluster.getSolrClient()),
           ImmutableMap.of("/config/boolVal", "true", "/config/strVal", "Something","/config/longVal", "1234" ));
 
-        p.strVal = "Something else";
+        cfg.strVal = "Something else";
         new V2Request.Builder("/cluster/plugin")
                 .forceV2(true)
                 .POST()
@@ -226,7 +228,7 @@ public class TestContainerPlugin extends SolrCloudTestCase {
                         .forceV2(true)
                         .GET()
                         .build().process(cluster.getSolrClient()),
-                ImmutableMap.of("/config/boolVal", "true", "/config/strVal", p.strVal,"/config/longVal", "1234" ));
+                ImmutableMap.of("/config/boolVal", "true", "/config/strVal", cfg.strVal,"/config/longVal", "1234" ));
 
         // kill the Overseer leader
       for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
@@ -391,12 +393,6 @@ public class TestContainerPlugin extends SolrCloudTestCase {
 
     @JsonProperty
     public Boolean boolVal;
-
-    @JsonProperty
-    public String name;
-
-    @JsonProperty(value = "class", required = true)
-    public String klass;
   }
 
   public static class C6 implements ClusterSingleton {
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/PluginMeta.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/PluginMeta.java
index 80098ca..5bee19f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/PluginMeta.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/PluginMeta.java
@@ -26,24 +26,33 @@ import org.apache.solr.common.util.ReflectMapWriter;
  * POJO for a plugin metadata used in container plugins
  */
 public class PluginMeta implements ReflectMapWriter {
+  /** Unique plugin name, required. */
   @JsonProperty(required = true)
   public String name;
 
+  /** Plugin implementation class, required. */
   @JsonProperty(value = "class", required = true)
   public String klass;
 
+  /** Plugin version. */
   @JsonProperty
   public String version;
 
+  /** Plugin API path prefix, optional. */
   @JsonProperty("path-prefix")
   public String pathPrefix;
 
+  /** Plugin configuration object, optional. */
+  @JsonProperty
+  public Object config;
+
 
   public PluginMeta copy() {
     PluginMeta result = new PluginMeta();
     result.name = name;
     result.klass = klass;
     result.version = version;
+    result.config = config;
     return result;
   }
 
@@ -53,7 +62,8 @@ public class PluginMeta implements ReflectMapWriter {
       PluginMeta that = (PluginMeta) obj;
       return Objects.equals(this.name, that.name) &&
           Objects.equals(this.klass, that.klass) &&
-          Objects.equals(this.version, that.version);
+          Objects.equals(this.version, that.version) &&
+          Objects.equals(this.config, that.config);
     }
     return false;
   }