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

[lucene-solr] branch jira/solr-15016 updated: SOLR-15016: Improve config hierarchy. Cleanup. Fix the discovery of config implementation class.

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

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


The following commit(s) were added to refs/heads/jira/solr-15016 by this push:
     new 2d8a1c3  SOLR-15016: Improve config hierarchy. Cleanup. Fix the discovery of config implementation class.
2d8a1c3 is described below

commit 2d8a1c389e729609969e381d99de4d13e336b49a
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Dec 3 14:26:10 2020 +0100

    SOLR-15016: Improve config hierarchy. Cleanup. Fix the discovery of
    config implementation class.
---
 .../apache/solr/api/ContainerPluginsRegistry.java  | 20 +++----
 .../impl/CollectionsRepairEventListener.java       |  3 +-
 ...ementConfig.java => PlacementPluginConfig.java} | 30 +++--------
 .../cluster/placement/PlacementPluginFactory.java  | 18 ++++++-
 .../impl/DelegatingPlacementPluginFactory.java     | 63 ++++++++++++++++++++++
 .../impl/PlacementPluginFactoryLoader.java         | 61 +++++----------------
 .../placement/plugins/AffinityPlacementConfig.java | 16 ++++--
 .../plugins/AffinityPlacementFactory.java          | 20 +------
 .../plugins/MinimizeCoresPlacementFactory.java     | 12 ++++-
 .../placement/plugins/RandomPlacementFactory.java  | 12 ++++-
 .../java/org/apache/solr/core/CoreContainer.java   | 14 ++---
 .../OverseerCollectionConfigSetProcessorTest.java  |  7 +++
 .../impl/PlacementPluginIntegrationTest.java       | 36 ++++++++-----
 .../client/solrj/request/beans/PluginMeta.java     |  3 +-
 14 files changed, 187 insertions(+), 128 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 9b5c6a1..453b092 100644
--- a/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
+++ b/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
@@ -181,9 +181,8 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
       } else {
         //ADDED or UPDATED
         PluginMetaHolder info = newState.get(e.getKey());
-        ApiInfo apiInfo = null;
         List<String> errs = new ArrayList<>();
-        apiInfo = new ApiInfo(info,errs);
+        ApiInfo apiInfo = new ApiInfo(info,errs);
         if (!errs.isEmpty()) {
           log.error(StrUtils.join(errs, ','));
           continue;
@@ -239,8 +238,7 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
 
   @SuppressWarnings({"rawtypes", "unchecked"})
   private static  Map<String, String> getTemplateVars(PluginMeta pluginMeta) {
-    Map result = makeMap("plugin-name", pluginMeta.name, "path-prefix", pluginMeta.pathPrefix);
-    return result;
+    return (Map) makeMap("plugin-name", pluginMeta.name, "path-prefix", pluginMeta.pathPrefix);
   }
 
   private static class ApiHolder extends Api {
@@ -273,7 +271,7 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
     private final PluginMetaHolder holder;
 
     @JsonProperty
-    private PluginMeta info;
+    private final PluginMeta info;
 
     @JsonProperty(value = "package")
     public final String pkg;
@@ -412,7 +410,8 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
 
   }
 
-  /**Get the generic type of a {@link ConfigurablePlugin}
+  /**
+   * Get the generic type of a {@link ConfigurablePlugin}
    */
   @SuppressWarnings("rawtypes")
   public static Class getConfigClass(ConfigurablePlugin<?> o) {
@@ -422,7 +421,10 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
       for (Type type : interfaces) {
         if (type instanceof ParameterizedType) {
           ParameterizedType parameterizedType = (ParameterizedType) type;
-          if (parameterizedType.getRawType() == ConfigurablePlugin.class) {
+          Type rawType = parameterizedType.getRawType();
+          if (rawType == ConfigurablePlugin.class ||
+              // or if a super interface is a ConfigurablePlugin
+              ((rawType instanceof Class) && ConfigurablePlugin.class.isAssignableFrom((Class) rawType))) {
             return (Class) parameterizedType.getActualTypeArguments()[0];
           }
         }
@@ -442,10 +444,10 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
   }
 
   public enum Diff {
-    ADDED, REMOVED, UNCHANGED, UPDATED;
+    ADDED, REMOVED, UNCHANGED, UPDATED
   }
 
-  public static Map<String, Diff> compareMaps(Map<String,? extends Object> a, Map<String,? extends Object> b) {
+  public static Map<String, Diff> compareMaps(Map<String, ?> a, Map<String, ?> b) {
     if(a.isEmpty() && b.isEmpty()) return null;
     Map<String, Diff> result = new HashMap<>(Math.max(a.size(), b.size()));
     a.forEach((k, v) -> {
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 a1dc136..fd5f610 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
@@ -42,6 +42,7 @@ 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.MapWriter;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ReplicaPosition;
@@ -77,7 +78,7 @@ public class CollectionsRepairEventListener implements ClusterEventListener, Clu
   private int waitForSecond = DEFAULT_WAIT_FOR_SEC;
 
   private ScheduledThreadPoolExecutor waitForExecutor;
-  private PlacementPluginFactory placementPluginFactory;
+  private PlacementPluginFactory<? extends MapWriter> placementPluginFactory;
 
   public CollectionsRepairEventListener(CoreContainer cc) {
     this.solrClient = cc.getSolrClientCache().getCloudSolrClient(cc.getZkController().getZkClient().getZkServerAddress());
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/PlacementPluginConfig.java
similarity index 54%
copy from solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementConfig.java
copy to solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginConfig.java
index dbfa564..fd5566d 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementConfig.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginConfig.java
@@ -14,33 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.solr.cluster.placement;
 
-package org.apache.solr.cluster.placement.plugins;
-
-import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.util.ReflectMapWriter;
 
 /**
- *
+ * Configuration beans should use this interface to define public
+ * (mutable) configuration properties. Implementations must have a
+ * public zero-args constructor. Class fields may be optionally
+ * annotated with {@link org.apache.solr.common.annotation.JsonProperty} if needed.
  */
-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;
-  }
+public interface PlacementPluginConfig extends ReflectMapWriter {
 }
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 fc537ca..bed9ea9 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
@@ -17,6 +17,8 @@
 
 package org.apache.solr.cluster.placement;
 
+import org.apache.solr.api.ConfigurablePlugin;
+
 /**
  * Factory implemented by client code and configured in container plugins allowing the creation of instances of
  * {@link PlacementPlugin} to be used for replica placement computation.
@@ -24,16 +26,28 @@ package org.apache.solr.cluster.placement;
  * {@link org.apache.solr.api.ConfigurablePlugin} with the appropriate configuration
  * bean type.</p>
  */
-public interface PlacementPluginFactory {
+public interface PlacementPluginFactory<T extends PlacementPluginConfig> extends ConfigurablePlugin<T> {
   /**
    * 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
+   * Returns an instance of the plugin that will be repeatedly (and concurrently) 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();
+
+  /**
+   * Return the configuration of the plugin.
+   */
+  T getConfig();
+
+  /**
+   * Useful for plugins that don't use any configuration.
+   */
+  class NoConfig implements PlacementPluginConfig {
+    public static NoConfig INSTANCE = new NoConfig();
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/DelegatingPlacementPluginFactory.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/DelegatingPlacementPluginFactory.java
new file mode 100644
index 0000000..b8c5214
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/DelegatingPlacementPluginFactory.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 org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementPluginConfig;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+
+/**
+ * Helper class to support dynamic reloading of plugin implementations.
+ */
+public final class DelegatingPlacementPluginFactory implements PlacementPluginFactory<PlacementPluginFactory.NoConfig> {
+
+  private volatile PlacementPluginFactory<? extends PlacementPluginConfig> delegate;
+  // support for tests to make sure the update is completed
+  private volatile int version;
+
+  @Override
+  public PlacementPlugin createPluginInstance() {
+    if (delegate != null) {
+      return delegate.createPluginInstance();
+    } else {
+      return null;
+    }
+  }
+
+  public void setDelegate(PlacementPluginFactory<? extends PlacementPluginConfig> delegate) {
+    this.delegate = delegate;
+    this.version++;
+  }
+
+  public PlacementPluginFactory<? extends PlacementPluginConfig> getDelegate() {
+    return delegate;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  @Override
+  public void configure(NoConfig cfg) {
+    // no-op
+  }
+
+  @Override
+  public NoConfig getConfig() {
+    return NoConfig.INSTANCE;
+  }
+}
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 523fa17..4348a0d 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,7 +18,8 @@
 package org.apache.solr.cluster.placement.impl;
 
 import org.apache.solr.api.ContainerPluginsRegistry;
-import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.cluster.placement.PlacementPluginConfig;
 import org.apache.solr.cluster.placement.PlacementPluginFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -32,11 +33,10 @@ import java.lang.invoke.MethodHandles;
 public class PlacementPluginFactoryLoader {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static PlacementPluginFactory load(ContainerPluginsRegistry plugins) {
-    final DelegatingPlacementPluginFactory pluginFactory = new DelegatingPlacementPluginFactory();
+  public static void load(DelegatingPlacementPluginFactory pluginFactory, ContainerPluginsRegistry plugins) {
     ContainerPluginsRegistry.ApiInfo pluginFactoryInfo = plugins.getPlugin(PlacementPluginFactory.PLUGIN_NAME);
     if (pluginFactoryInfo != null && (pluginFactoryInfo.getInstance() instanceof PlacementPluginFactory)) {
-      pluginFactory.setDelegate((PlacementPluginFactory) pluginFactoryInfo.getInstance());
+      pluginFactory.setDelegate((PlacementPluginFactory<? extends PlacementPluginConfig>) pluginFactoryInfo.getInstance());
     }
     ContainerPluginsRegistry.PluginRegistryListener pluginListener = new ContainerPluginsRegistry.PluginRegistryListener() {
       @Override
@@ -46,11 +46,7 @@ public class PlacementPluginFactoryLoader {
         }
         Object instance = plugin.getInstance();
         if (instance instanceof PlacementPluginFactory) {
-          if (PlacementPluginFactory.PLUGIN_NAME.equals(plugin.getInfo().name)) {
-            pluginFactory.setDelegate((PlacementPluginFactory) instance);
-          } else {
-            log.warn("Ignoring PlacementPluginFactory plugin with non-standard name: {}", plugin.getInfo());
-          }
+          setDelegate(plugin.getInfo(), instance);
         }
       }
 
@@ -61,53 +57,24 @@ public class PlacementPluginFactoryLoader {
         }
         Object instance = plugin.getInstance();
         if (instance instanceof PlacementPluginFactory) {
-          if (PlacementPluginFactory.PLUGIN_NAME.equals(plugin.getInfo().name)) {
-            pluginFactory.setDelegate(null);
-          } else {
-            log.warn("Ignoring PlacementPluginFactory plugin with non-standard name: {}", plugin.getInfo());
-          }
+          setDelegate(plugin.getInfo(), null);
         }
       }
 
       @Override
       public void modified(ContainerPluginsRegistry.ApiInfo old, ContainerPluginsRegistry.ApiInfo replacement) {
-        deleted(old);
         added(replacement);
       }
+
+      private void setDelegate(PluginMeta pluginMeta, Object instance) {
+        if (PlacementPluginFactory.PLUGIN_NAME.equals(pluginMeta.name)) {
+          pluginFactory.setDelegate((PlacementPluginFactory<? extends PlacementPluginConfig>) instance);
+        } else {
+          log.warn("Ignoring PlacementPluginFactory plugin with non-standard name: {}", pluginMeta);
+        }
+      }
     };
     plugins.registerListener(pluginListener);
-    return pluginFactory;
   }
 
-  /**
-   * Helper class to support dynamic reloading of plugin implementations.
-   */
-  public static final class DelegatingPlacementPluginFactory implements PlacementPluginFactory {
-
-    private PlacementPluginFactory delegate;
-    // support for tests to make sure the update is completed
-    private int version;
-
-    @Override
-    public PlacementPlugin createPluginInstance() {
-      if (delegate != null) {
-        return delegate.createPluginInstance();
-      } else {
-        return null;
-      }
-    }
-
-    public void setDelegate(PlacementPluginFactory delegate) {
-      this.delegate = delegate;
-      this.version++;
-    }
-
-    public PlacementPluginFactory getDelegate() {
-      return delegate;
-    }
-
-    public int getVersion() {
-      return version;
-    }
-  }
 }
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
index dbfa564..bbf8dc8 100644
--- 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
@@ -17,19 +17,29 @@
 
 package org.apache.solr.cluster.placement.plugins;
 
+import org.apache.solr.cluster.placement.PlacementPluginConfig;
 import org.apache.solr.common.annotation.JsonProperty;
-import org.apache.solr.common.util.ReflectMapWriter;
 
 /**
- *
+ * Configuration bean for {@link AffinityPlacementFactory}.
  */
-public class AffinityPlacementConfig implements ReflectMapWriter {
+public class AffinityPlacementConfig implements PlacementPluginConfig {
 
   public static final AffinityPlacementConfig DEFAULT = new AffinityPlacementConfig();
 
+  /**
+   * If a node has strictly less GB of free disk than this value, the node is excluded from assignment decisions.
+   * Set to 0 or less to disable.
+   */
   @JsonProperty
   public long minimalFreeDiskGB;
 
+  /**
+   * Replica allocation will assign replicas to nodes with at least this number of GB of free disk space regardless
+   * of the number of cores on these nodes rather than assigning replicas to nodes with less than this amount of free
+   * disk space if that's an option (if that's not an option, replicas can still be assigned to nodes with less than this
+   * amount of free space).
+   */
   @JsonProperty
   public long 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 443b5b2..be72190 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
@@ -17,10 +17,8 @@
 
 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;
@@ -117,7 +115,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, ConfigurablePlugin<AffinityPlacementConfig> {
+public class AffinityPlacementFactory implements PlacementPluginFactory<AffinityPlacementConfig> {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   /**
@@ -142,20 +140,6 @@ public class AffinityPlacementFactory implements PlacementPluginFactory, Configu
    */
   public static final String UNDEFINED_AVAILABILITY_ZONE = "uNd3f1NeD";
 
-  /**
-   * If a node has strictly less GB of free disk than this value, the node is excluded from assignment decisions.
-   * Set to 0 or less to disable.
-   */
-  public static final String MINIMAL_FREE_DISK_GB = "minimalFreeDiskGB";
-
-  /**
-   * Replica allocation will assign replicas to nodes with at least this number of GB of free disk space regardless
-   * of the number of cores on these nodes rather than assigning replicas to nodes with less than this amount of free
-   * disk space if that's an option (if that's not an option, replicas can still be assigned to nodes with less than this
-   * amount of free space).
-   */
-  public static final String PRIORITIZED_FREE_DISK_GB = "prioritizedFreeDiskGB";
-
   private AffinityPlacementConfig config = AffinityPlacementConfig.DEFAULT;
 
   /**
@@ -178,7 +162,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory, Configu
     this.config = cfg;
   }
 
-  @VisibleForTesting
+  @Override
   public AffinityPlacementConfig getConfig() {
     return config;
   }
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 d6f88f3..f0a2b6e 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
@@ -40,13 +40,23 @@ import org.apache.solr.common.util.SuppressForbidden;
  *
  * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
  */
-public class MinimizeCoresPlacementFactory implements PlacementPluginFactory {
+public class MinimizeCoresPlacementFactory implements PlacementPluginFactory<PlacementPluginFactory.NoConfig> {
 
   @Override
   public PlacementPlugin createPluginInstance() {
     return new MinimizeCoresPlacementPlugin();
   }
 
+  @Override
+  public void configure(NoConfig cfg) {
+    // no-op
+  }
+
+  @Override
+  public NoConfig getConfig() {
+    return NoConfig.INSTANCE;
+  }
+
   static private class MinimizeCoresPlacementPlugin implements PlacementPlugin {
 
     @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
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 cec1d9b..9829e6c 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
@@ -36,13 +36,23 @@ import org.apache.solr.cluster.placement.*;
  *
  * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
  */
-public class RandomPlacementFactory implements PlacementPluginFactory {
+public class RandomPlacementFactory implements PlacementPluginFactory<PlacementPluginFactory.NoConfig> {
 
   @Override
   public PlacementPlugin createPluginInstance() {
     return new RandomPlacementPlugin();
   }
 
+  @Override
+  public void configure(NoConfig cfg) {
+
+  }
+
+  @Override
+  public NoConfig getConfig() {
+    return NoConfig.INSTANCE;
+  }
+
   public static class RandomPlacementPlugin implements PlacementPlugin {
     private Random random = new Random();
 
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 f31dfa7..346f769 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -75,8 +75,10 @@ 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.PlacementPluginFactory;
+import org.apache.solr.cluster.placement.impl.DelegatingPlacementPluginFactory;
 import org.apache.solr.cluster.placement.impl.PlacementPluginFactoryLoader;
 import org.apache.solr.common.AlreadyClosedException;
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.DocCollection;
@@ -257,9 +259,8 @@ public class CoreContainer {
           !getZkController().getOverseer().isClosed(),
       (r) -> this.runAsync(r));
 
-  // initially these are the same to collect the plugin-based listeners during init
-  private ClusterEventProducer clusterEventProducer;
-  private PlacementPluginFactory placementPluginFactory;
+  private volatile ClusterEventProducer clusterEventProducer;
+  private final DelegatingPlacementPluginFactory placementPluginFactory = new DelegatingPlacementPluginFactory();
 
   private PackageStoreAPI packageStoreAPI;
   private PackageLoader packageLoader;
@@ -899,8 +900,9 @@ public class CoreContainer {
       containerHandlers.getApiBag().registerObject(containerPluginsApi.readAPI);
       containerHandlers.getApiBag().registerObject(containerPluginsApi.editAPI);
 
-      // get the placement plugin
-      placementPluginFactory = PlacementPluginFactoryLoader.load(containerPluginsRegistry);
+      // initialize the placement plugin factory wrapper
+      // with the plugin configuration from the registry
+      PlacementPluginFactoryLoader.load(placementPluginFactory, containerPluginsRegistry);
 
       // create target ClusterEventProducer (possibly from plugins)
       clusterEventProducer = clusterEventProducerFactory.create(containerPluginsRegistry);
@@ -2186,7 +2188,7 @@ public class CoreContainer {
     return clusterEventProducer;
   }
 
-  public PlacementPluginFactory getPlacementPluginFactory() {
+  public PlacementPluginFactory<? extends MapWriter> getPlacementPluginFactory() {
     return placementPluginFactory;
   }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
index c811bf8..694ad4a 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
@@ -42,6 +42,8 @@ import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.cloud.Overseer.LeaderStatus;
 import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+import org.apache.solr.cluster.placement.impl.DelegatingPlacementPluginFactory;
 import org.apache.solr.common.cloud.Aliases;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
@@ -121,6 +123,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
   private static CoreContainer coreContainerMock;
   private static UpdateShardHandler updateShardHandlerMock;
   private static HttpClient httpClientMock;
+  private static PlacementPluginFactory placementPluginFactoryMock;
   
   private static ObjectCache objectCache;
   private Map<String, byte[]> zkClientData = new HashMap<>();
@@ -180,6 +183,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     coreContainerMock = mock(CoreContainer.class);
     updateShardHandlerMock = mock(UpdateShardHandler.class);
     httpClientMock = mock(HttpClient.class);
+    placementPluginFactoryMock = mock(PlacementPluginFactory.class);
   }
   
   @AfterClass
@@ -204,6 +208,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     coreContainerMock = null;
     updateShardHandlerMock = null;
     httpClientMock = null;
+    placementPluginFactoryMock = null;
   }
   
   @Before
@@ -233,6 +238,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     reset(coreContainerMock);
     reset(updateShardHandlerMock);
     reset(httpClientMock);
+    reset(placementPluginFactoryMock);
 
     zkClientData.clear();
     collectionsSet.clear();
@@ -361,6 +367,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     when(overseerMock.getSolrCloudManager()).thenReturn(cloudDataProviderMock);
     when(overseerMock.getCoreContainer()).thenReturn(coreContainerMock);
     when(coreContainerMock.getUpdateShardHandler()).thenReturn(updateShardHandlerMock);
+    when(coreContainerMock.getPlacementPluginFactory()).thenReturn(placementPluginFactoryMock);
     when(updateShardHandlerMock.getDefaultHttpClient()).thenReturn(httpClientMock);
     
     when(zkControllerMock.getSolrCloudManager()).thenReturn(cloudDataProviderMock);
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 6159358..f17ef00 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
@@ -29,17 +29,21 @@ import org.apache.solr.cluster.placement.plugins.AffinityPlacementConfig;
 import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
 import org.apache.solr.cloud.MiniSolrCloudCluster;
 import org.apache.solr.cluster.placement.plugins.MinimizeCoresPlacementFactory;
-import org.apache.solr.common.cloud.ClusterProperties;
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreContainer;
+import org.apache.solr.util.LogLevel;
 import org.apache.solr.util.TimeOut;
 
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import java.lang.invoke.MethodHandles;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
@@ -52,11 +56,12 @@ import static java.util.Collections.singletonMap;
 /**
  * Test for {@link MinimizeCoresPlacementFactory} using a {@link MiniSolrCloudCluster}.
  */
+@LogLevel("org.apache.solr.cluster.placement.impl=DEBUG")
 public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static final String COLLECTION = PlacementPluginIntegrationTest.class.getName() + "_collection";
 
-  private static ClusterProperties clusterProperties;
   private static SolrCloudManager cloudManager;
   private static CoreContainer cc;
 
@@ -69,7 +74,6 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
         .configure();
     cc = cluster.getJettySolrRunner(0).getCoreContainer();
     cloudManager = cc.getZkController().getSolrCloudManager();
-    clusterProperties = new ClusterProperties(cluster.getZkClient());
   }
 
   @After
@@ -111,9 +115,7 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
     DocCollection collection = clusterState.getCollectionOrNull(COLLECTION);
     assertNotNull(collection);
     Map<String, AtomicInteger> coresByNode = new HashMap<>();
-    collection.forEachReplica((shard, replica) -> {
-      coresByNode.computeIfAbsent(replica.getNodeName(), n -> new AtomicInteger()).incrementAndGet();
-    });
+    collection.forEachReplica((shard, replica) -> coresByNode.computeIfAbsent(replica.getNodeName(), n -> new AtomicInteger()).incrementAndGet());
     int maxCores = 0;
     int minCores = Integer.MAX_VALUE;
     for (Map.Entry<String, AtomicInteger> entry : coresByNode.entrySet()) {
@@ -131,7 +133,15 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
   }
 
   @Test
+  @SuppressWarnings("unchecked")
   public void testDynamicReconfiguration() throws Exception {
+    PlacementPluginFactory<? extends MapWriter> pluginFactory = cc.getPlacementPluginFactory();
+    assertTrue("wrong type " + pluginFactory.getClass().getName(), pluginFactory instanceof DelegatingPlacementPluginFactory);
+    DelegatingPlacementPluginFactory wrapper = (DelegatingPlacementPluginFactory) pluginFactory;
+
+    int version = wrapper.getVersion();
+    log.debug("--initial version={}", version);
+
     PluginMeta plugin = new PluginMeta();
     plugin.name = PlacementPluginFactory.PLUGIN_NAME;
     plugin.klass = MinimizeCoresPlacementFactory.class.getName();
@@ -142,13 +152,10 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
         .build();
     req.process(cluster.getSolrClient());
 
-    PlacementPluginFactory pluginFactory = cc.getPlacementPluginFactory();
-    assertTrue("wrong type " + pluginFactory.getClass().getName(), pluginFactory instanceof PlacementPluginFactoryLoader.DelegatingPlacementPluginFactory);
-    PlacementPluginFactoryLoader.DelegatingPlacementPluginFactory wrapper = (PlacementPluginFactoryLoader.DelegatingPlacementPluginFactory) pluginFactory;
-    // should already have some updates
-    int version = wrapper.getVersion();
+    version = waitForVersionChange(version, wrapper, 10);
+
     assertTrue("wrong version " + version, version > 0);
-    PlacementPluginFactory factory = wrapper.getDelegate();
+    PlacementPluginFactory<? extends MapWriter> factory = wrapper.getDelegate();
     assertTrue("wrong type " + factory.getClass().getName(), factory instanceof MinimizeCoresPlacementFactory);
 
     // reconfigure
@@ -208,12 +215,12 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
         .withPayload("{remove: '" + PlacementPluginFactory.PLUGIN_NAME + "'}")
         .build();
     req.process(cluster.getSolrClient());
-    version = waitForVersionChange(version, wrapper, 10);
+    waitForVersionChange(version, wrapper, 10);
     factory = wrapper.getDelegate();
     assertNull("no factory should be present", factory);
   }
 
-  private int waitForVersionChange(int currentVersion, PlacementPluginFactoryLoader.DelegatingPlacementPluginFactory wrapper, int timeoutSec) throws Exception {
+  private int waitForVersionChange(int currentVersion, DelegatingPlacementPluginFactory wrapper, int timeoutSec) throws Exception {
     TimeOut timeout = new TimeOut(timeoutSec, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     while (!timeout.hasTimedOut()) {
@@ -222,6 +229,7 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
         throw new Exception("Invalid version - went back! currentVersion=" + currentVersion +
             " newVersion=" + newVersion);
       } else if (currentVersion < newVersion) {
+        log.debug("--current version was {}, new version is {}", currentVersion, newVersion);
         return newVersion;
       }
       timeout.sleep(200);
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 3586ffa..bab68b2 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
@@ -21,7 +21,6 @@ import java.util.Objects;
 
 import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.util.ReflectMapWriter;
-import org.apache.solr.common.util.Utils;
 
 /**
  * POJO for a plugin metadata used in container plugins
@@ -75,6 +74,6 @@ public class PluginMeta implements ReflectMapWriter {
 
   @Override
   public String toString() {
-    return Utils.toJSONString(this);
+    return jsonStr();
   }
 }