You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2021/03/26 16:41:42 UTC

[lucene] tag history/branches/lucene-solr/jira/solr-15016 created (now 6d9e22c)

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

janhoy pushed a change to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git.


      at 6d9e22c  (commit)
This tag includes the following new commits:

     new 63607f4  SOLR-15004: Unit test for SimpleClusterAbstractions.
     new 2267f3d  solr-15004: initial example (WIP and buggy) of plugin test
     new 023ebc3  SOLR-15004: fix failure in SamplePluginAffinityReplicaPlacementTest
     new a6a47a0  SOLR-15004: Add integration test. Move some properties so that they are accessible (and can be validated) in ClusterProperties. Fix a bug in update().
     new be681a3  Inverted the plugin and factory class hierarchy for each sample plugin
     new 1d50edb  SOLR-15004: Add a scalability test.
     new 0cfcf83  Move plugin Javadoc from private plugin to public factory
     new 9ca5920  SOLR-15004: Start testing the availability zone affinity. Add toString() to make it easier to debug / report. Improve test abstractions to make it easier to set up tests.
     new bd63ad1  Introduce a builder class for unit tests to simplify creating cluster states and collections
     new 0a112f5  Merge branch 'jira/solr-15004' of https://github.com/apache/lucene-solr into jira/solr-15004 (and make precommit happy)
     new cc52210  SOLR-15004: Extend the NodeBuilder, move testAffinity to use the Builders.
     new 20a2778  SOLR-15004: Fix the use of prefixes for sysprops and metrics. Finish the affinity test.
     new 374fd7f  SOLR-15004: Test replica types placement.
     new dfb0eb5  Have each replica type provide the corresponding suffix char to use in replica names
     new 8b06bbb  Reformat to correct tab length (+ some other IntelliJ magic)
     new 2be41f3  Remove PluginTestHelper, use Builders instead. Add getShardNames() to SolrCollection
     new d0e989c  SOLR-15004: Move around some of the test support classes so that they can be used in other placement tests (this required changing visibility of some classes / methods). Allow using the test framework random() for repeatable randomness.
     new 5705765  SOLR-15004: Some renaming to clarify the purpose of plugin config params.
     new 0a59e11  Rename a couple of leftover deprioritizedFreeDiskGB into prioritizedFreeDiskGB
     new 8450ea8  Reformat placement plugin code to right tab size
     new 4d08c71  SOLR-15016: Initial changes. These also include changes in PR 2099.
     new 29fd3f9  SOLR-15016: Remove unneeded file.
     new 6853640  SOLR-15016: Add unit test for dynamic reconfiguration. Remove old API endpoint.
     new 0cdd738  Merge branch 'master' into jira/solr-15016
     new 88083a4  SOLR-15016: Use '.' prefix for predefined plugins. Prevent errors when using unquoted JSON with string with a leading dot :) Make sure to process only plugin configs with the predefined names.
     new 947db7a  SOLR-15016: Fix precommit issues.
     new 2d8a1c3  SOLR-15016: Improve config hierarchy. Cleanup. Fix the discovery of config implementation class.
     new 9a2865b  SOLR-15016: Fix issues found in review.
     new 0181f85  SOLR-15016: Fix more left-overs from hierarchy cleanup.
     new 60c8fb3  SOLR-15016: More fixes after review.
     new b5e5dda  SOLR-15016: Use utility method here to allow for nulls.
     new 345db0e4 Merge branch 'master' into jira/solr-15016
     new 6d9e22c  SOLR-15016: CHANGES.txt.

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


[lucene] 14/33: Have each replica type provide the corresponding suffix char to use in replica names

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit dfb0eb51e3edf62673dc69bef4e61507a65f495b
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Tue Nov 24 18:58:06 2020 +0100

    Have each replica type provide the corresponding suffix char to use in replica names
---
 solr/core/src/java/org/apache/solr/cluster/Replica.java      | 12 +++++++++++-
 .../org/apache/solr/cluster/placement/impl/Builders.java     |  2 +-
 .../apache/solr/cluster/placement/impl/PluginTestHelper.java |  6 +-----
 3 files changed, 13 insertions(+), 7 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cluster/Replica.java b/solr/core/src/java/org/apache/solr/cluster/Replica.java
index 2c9230f..ff83f3f 100644
--- a/solr/core/src/java/org/apache/solr/cluster/Replica.java
+++ b/solr/core/src/java/org/apache/solr/cluster/Replica.java
@@ -42,7 +42,17 @@ public interface Replica {
    * The order of this enum is important from the most to least "important" replica type.
    */
   enum ReplicaType {
-    NRT, TLOG, PULL
+    NRT('n'), TLOG('t'), PULL('p');
+
+    private char suffixChar;
+
+    ReplicaType(char suffixChar) {
+      this.suffixChar = suffixChar;
+    }
+
+    public char getSuffixChar() {
+      return suffixChar;
+    }
   }
 
   enum ReplicaState {
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
index aa5092b..bbbbc0e 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
@@ -144,7 +144,7 @@ public class Builders {
                 for (Pair<Replica.ReplicaType, Integer> tc : replicaTypes) {
                     Replica.ReplicaType type = tc.first();
                     int count = tc.second();
-                    String replicaPrefix = collectionName + "_" + shardName + "_replica_" + type.name().toLowerCase(Locale.ROOT).charAt(0);
+                    String replicaPrefix = collectionName + "_" + shardName + "_replica_" + type.getSuffixChar();
                     for (int r = 0; r < count; r++) {
                         String replicaName = replicaPrefix + r;
                         String coreName = replicaName + "_c";
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
index f8b4d87..5b88b9d 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
@@ -60,18 +60,14 @@ public class PluginTestHelper {
             int totalReplicas = nrtReplicas + tlogReplicas + pullReplicas;
             for (int r = 0; r < totalReplicas; r++) {
                 Replica.ReplicaType type;
-                String suffix;
                 if (r < nrtReplicas) {
                     type = Replica.ReplicaType.NRT;
-                    suffix = "n";
                 } else if (r < nrtReplicas + tlogReplicas) {
                     type = Replica.ReplicaType.TLOG;
-                    suffix = "t";
                 } else {
                     type = Replica.ReplicaType.PULL;
-                    suffix = "p";
                 }
-                String replicaName = shardName + "_replica_" + suffix + r;
+                String replicaName = shardName + "_replica_" + type.getSuffixChar() + r;
                 String coreName = replicaName + "_c";
                 final Node node;
                 if (!nodeIterator.hasNext()) {

[lucene] 17/33: SOLR-15004: Move around some of the test support classes so that they can be used in other placement tests (this required changing visibility of some classes / methods). Allow using the test framework random() for repeatable randomness.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit d0e989c52e215460d3a1faa426b3724ed30c3b4a
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Wed Nov 25 14:13:05 2020 +0100

    SOLR-15004: Move around some of the test support classes so that they can be used in other
    placement tests (this required changing visibility of some classes / methods). Allow using
    the test framework random() for repeatable randomness.
---
 .../placement/impl/AttributeFetcherImpl.java       |  4 +-
 .../placement/impl/AttributeValuesImpl.java        |  2 +-
 .../placement/impl/PlacementPlanFactoryImpl.java   |  2 +-
 .../placement/impl/PlacementPluginConfigImpl.java  |  2 +-
 .../placement/impl/PlacementRequestImpl.java       |  2 +-
 .../plugins/AffinityPlacementFactory.java          | 31 +++++----
 .../placement/plugins/RandomPlacementFactory.java  | 11 +++-
 .../{impl => }/AttributeFetcherForTest.java        |  2 +-
 .../cluster/placement/{impl => }/Builders.java     | 76 +++++++++++-----------
 .../{impl => }/ClusterAbstractionsForTest.java     |  2 +-
 .../AffinityPlacementFactoryTest.java              | 12 ++--
 11 files changed, 83 insertions(+), 63 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeFetcherImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeFetcherImpl.java
index 98367d3..97d4ba1 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeFetcherImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeFetcherImpl.java
@@ -218,11 +218,11 @@ public class AttributeFetcherImpl implements AttributeFetcher {
         }
     }
 
-    static String getMetricSnitchTag(String metricName, NodeMetricRegistry registry) {
+    public static String getMetricSnitchTag(String metricName, NodeMetricRegistry registry) {
         return SolrClientNodeStateProvider.METRICS_PREFIX + SolrMetricManager.getRegistryName(getGroupFromMetricRegistry(registry), metricName);
     }
 
-    static String getSystemPropertySnitchTag(String name) {
+    public static String getSystemPropertySnitchTag(String name) {
         return ImplicitSnitch.SYSPROP + name;
     }
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeValuesImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeValuesImpl.java
index 78c2143..0f4c1ab 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeValuesImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeValuesImpl.java
@@ -34,7 +34,7 @@ public class AttributeValuesImpl implements AttributeValues {
     final Map<String, Map<Node, String>> syspropSnitchToNodeToValue;
     final Map<String, Map<Node, Double>> metricSnitchToNodeToValue;
 
-    AttributeValuesImpl(Map<Node, Integer> nodeToCoreCount,
+    public AttributeValuesImpl(Map<Node, Integer> nodeToCoreCount,
                         Map<Node, AttributeFetcher.DiskHardwareType> nodeToDiskType,
                         Map<Node, Long> nodeToFreeDisk,
                         Map<Node, Long> nodeToTotalDisk,
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 3829372..56071da 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,7 +24,7 @@ import org.apache.solr.cluster.placement.*;
 
 import java.util.Set;
 
-class PlacementPlanFactoryImpl implements PlacementPlanFactory {
+public class PlacementPlanFactoryImpl implements PlacementPlanFactory {
     @Override
     public PlacementPlan createPlacementPlan(PlacementRequest request, Set<ReplicaPlacement> replicaPlacements) {
         return new PlacementPlanImpl(request, 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
index 4c079d4..98d02b7 100644
--- 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
@@ -112,7 +112,7 @@ public class PlacementPluginConfigImpl implements PlacementPluginConfig {
    *
    * <p>See configuration example and how-to in {@link AffinityPlacementFactory}.</p>
    */
-  static PlacementPluginConfig createConfigFromProperties(Map<String, Object> pluginConfig) {
+  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<>();
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
index 82a26d1..555b103 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
@@ -34,7 +34,7 @@ public class PlacementRequestImpl implements PlacementRequest {
   private final Set<Node> targetNodes;
   private final EnumMap<Replica.ReplicaType, Integer> countReplicas = new EnumMap<>(Replica.ReplicaType.class);
 
-  PlacementRequestImpl(SolrCollection solrCollection,
+  public PlacementRequestImpl(SolrCollection solrCollection,
                                Set<String> shardNames, Set<Node> targetNodes,
                                int countNrtReplicas, int countTlogReplicas, int countPullReplicas) {
     this.solrCollection = solrCollection;
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 6982b8f..ddddfd7 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,7 +17,9 @@
 
 package org.apache.solr.cluster.placement.plugins;
 
-import com.google.common.collect.*;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Ordering;
+import com.google.common.collect.TreeMultimap;
 import org.apache.solr.cluster.*;
 import org.apache.solr.cluster.placement.*;
 import org.apache.solr.common.util.Pair;
@@ -157,7 +159,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
    * See {@link AffinityPlacementFactory} for instructions on how to configure a cluster to use this plugin and details
    * on what the plugin does.
    */
-  static private class AffinityPlacementPlugin implements PlacementPlugin {
+  static class AffinityPlacementPlugin implements PlacementPlugin {
 
     /**
      * If a node has strictly less GB of free disk than this value, the node is excluded from assignment decisions.
@@ -173,6 +175,8 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
      */
     private final long deprioritizedFreeDiskGB;
 
+    private Random random = new Random();
+
     /**
      * The factory has decoded the configuration for the plugin instance and passes it the parameters it needs.
      */
@@ -181,6 +185,11 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
       this.deprioritizedFreeDiskGB = deprioritizedFreeDiskGB;
     }
 
+    @VisibleForTesting
+    void setRandom(Random random) {
+      this.random = random;
+    }
+
     @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
     public PlacementPlan computePlacement(Cluster cluster, PlacementRequest request, AttributeFetcher attributeFetcher,
                                           PlacementPlanFactory placementPlanFactory) throws PlacementException {
@@ -205,7 +214,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
       // All available zones of live nodes. Due to some nodes not being candidates for placement, and some existing replicas
       // being one availability zones that might be offline (i.e. their nodes are not live), this set might contain zones
       // on which it is impossible to place replicas. That's ok.
-      ImmutableSet<String> availabilityZones = getZonesFromNodes(nodes, attrValues);
+      Set<String> availabilityZones = getZonesFromNodes(nodes, attrValues);
 
       // Build the replica placement decisions here
       Set<ReplicaPlacement> replicaPlacements = new HashSet<>();
@@ -225,14 +234,14 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
       return placementPlanFactory.createPlacementPlan(request, replicaPlacements);
     }
 
-    private ImmutableSet<String> getZonesFromNodes(Set<Node> nodes, final AttributeValues attrValues) {
+    private Set<String> getZonesFromNodes(Set<Node> nodes, final AttributeValues attrValues) {
       Set<String> azs = new HashSet<>();
 
       for (Node n : nodes) {
         azs.add(getNodeAZ(n, attrValues));
       }
 
-      return ImmutableSet.copyOf(azs);
+      return Collections.unmodifiableSet(azs);
     }
 
     /**
@@ -247,7 +256,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
 
     /**
      * This class captures an availability zone and the nodes that are legitimate targets for replica placement in that
-     * Availability Zone. Instances are used as values in a {@link TreeMap} in which the total number of already
+     * Availability Zone. Instances are used as values in a {@link java.util.TreeMap} in which the total number of already
      * existing replicas in the AZ is the key. This allows easily picking the set of nodes from which to select a node for
      * placement in order to balance the number of replicas per AZ. Picking one of the nodes from the set is done using
      * different criteria unrelated to the Availability Zone (picking the node is based on the {@link CoresAndDiskComparator}
@@ -280,7 +289,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
      */
     private Pair<EnumMap<Replica.ReplicaType, Set<Node>>, Map<Node, Integer>> getNodesPerReplicaType(Set<Node> nodes, final AttributeValues attrValues) {
       EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes = new EnumMap<>(Replica.ReplicaType.class);
-      Map<Node, Integer> coresOnNodes = Maps.newHashMap();
+      Map<Node, Integer> coresOnNodes = new HashMap<>();
 
       for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
         replicaTypeToNodes.put(replicaType, new HashSet<>());
@@ -347,7 +356,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
      * </ol>
      */
     @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
-    private void makePlacementDecisions(SolrCollection solrCollection, String shardName, ImmutableSet<String> availabilityZones,
+    private void makePlacementDecisions(SolrCollection solrCollection, String shardName, Set<String> availabilityZones,
                                         Replica.ReplicaType replicaType, int numReplicas, final AttributeValues attrValues,
                                         EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes, Map<Node, Integer> coresOnNodes,
                                         PlacementPlanFactory placementPlanFactory, Set<ReplicaPlacement> replicaPlacements) throws PlacementException {
@@ -356,7 +365,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
 
       // Count existing replicas per AZ. We count only instances the type of replica for which we need to do placement. This
       // can be changed in the loop below if we want to count all replicas for the shard.
-      Map<String, Integer> azToNumReplicas = Maps.newHashMap();
+      Map<String, Integer> azToNumReplicas = new HashMap<>();
       // Add all "interesting" AZ's, i.e. AZ's for which there's a chance we can do placement.
       for (String az : availabilityZones) {
         azToNumReplicas.put(az, 0);
@@ -391,7 +400,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
       // (or try to place) replicas on AZ's that have fewer replicas
 
       // Get the candidate nodes per AZ in order to build (further down) a mapping of AZ to placement candidates.
-      Map<String, List<Node>> nodesPerAz = Maps.newHashMap();
+      Map<String, List<Node>> nodesPerAz = new HashMap<>();
       for (Node node : candidateNodes) {
         String nodeAz = getNodeAZ(node, attrValues);
         List<Node> nodesForAz = nodesPerAz.computeIfAbsent(nodeAz, k -> new ArrayList<>());
@@ -450,7 +459,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
           // unnecessary imbalance).
           // For example, if all nodes have 0 cores and same amount of free disk space, ideally we want to pick a random node
           // for placement, not always the same one due to some internal ordering.
-          Collections.shuffle(nodes, new Random());
+          Collections.shuffle(nodes, random);
 
           // Sort by increasing number of cores but pushing nodes with low free disk space to the end of the list
           nodes.sort(coresAndDiskComparator);
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 b9d3884..27699d8 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
@@ -23,6 +23,7 @@ import java.util.HashSet;
 import java.util.Random;
 import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.solr.cluster.Cluster;
 import org.apache.solr.cluster.Node;
 import org.apache.solr.cluster.Replica;
@@ -42,7 +43,13 @@ public class RandomPlacementFactory implements PlacementPluginFactory {
     return new RandomPlacementPlugin();
   }
 
-  static private class RandomPlacementPlugin implements PlacementPlugin {
+  public static class RandomPlacementPlugin implements PlacementPlugin {
+    private Random random = new Random();
+
+    @VisibleForTesting
+    public void setRandom(Random random) {
+      this.random = random;
+    }
 
     public PlacementPlan computePlacement(Cluster cluster, PlacementRequest request, AttributeFetcher attributeFetcher,
                                           PlacementPlanFactory placementPlanFactory) throws PlacementException {
@@ -61,7 +68,7 @@ public class RandomPlacementFactory implements PlacementPluginFactory {
       for (String shardName : request.getShardNames()) {
         // Shuffle the nodes for each shard so that replicas for a shard are placed on distinct yet random nodes
         ArrayList<Node> nodesToAssign = new ArrayList<>(cluster.getLiveNodes());
-        Collections.shuffle(nodesToAssign, new Random());
+        Collections.shuffle(nodesToAssign, random);
 
         for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
           placeForReplicaType(request.getCollection(), nodesToAssign, placementPlanFactory, replicaPlacements, shardName, request, rt);
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/AttributeFetcherForTest.java
similarity index 98%
rename from solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java
rename to solr/core/src/test/org/apache/solr/cluster/placement/AttributeFetcherForTest.java
index f053d1b..b1cc2a0 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/AttributeFetcherForTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.solr.cluster.placement.impl;
+package org.apache.solr.cluster.placement;
 
 import org.apache.solr.cluster.Node;
 import org.apache.solr.cluster.placement.AttributeFetcher;
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java b/solr/core/src/test/org/apache/solr/cluster/placement/Builders.java
similarity index 81%
rename from solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
rename to solr/core/src/test/org/apache/solr/cluster/placement/Builders.java
index 1880374..f10135a 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/Builders.java
@@ -1,8 +1,8 @@
-package org.apache.solr.cluster.placement.impl;
+package org.apache.solr.cluster.placement;
 
 import org.apache.solr.cluster.*;
-import org.apache.solr.cluster.placement.AttributeFetcher;
-import org.apache.solr.cluster.placement.AttributeValues;
+import org.apache.solr.cluster.placement.impl.AttributeFetcherImpl;
+import org.apache.solr.cluster.placement.impl.AttributeValuesImpl;
 import org.apache.solr.common.util.Pair;
 
 import java.util.*;
@@ -20,11 +20,11 @@ public class Builders {
     return new CollectionBuilder(collectionName);
   }
 
-  static class ClusterBuilder {
+  public static class ClusterBuilder {
     private LinkedList<NodeBuilder> nodeBuilders = new LinkedList<>();
     private LinkedList<CollectionBuilder> collectionBuilders = new LinkedList<>();
 
-    ClusterBuilder initializeNodes(int countNodes) {
+    public ClusterBuilder initializeNodes(int countNodes) {
       nodeBuilders = new LinkedList<>();
       for (int n = 0; n < countNodes; n++) {
         nodeBuilders.add(new NodeBuilder().setNodeName("node_" + n)); // Default name, can be changed
@@ -32,21 +32,21 @@ public class Builders {
       return this;
     }
 
-    LinkedList<NodeBuilder> getNodeBuilders() {
+    public LinkedList<NodeBuilder> getNodeBuilders() {
       return nodeBuilders;
     }
 
-    ClusterBuilder addCollection(CollectionBuilder collectionBuilder) {
+    public ClusterBuilder addCollection(CollectionBuilder collectionBuilder) {
       collectionBuilders.add(collectionBuilder);
       return this;
     }
 
-    Cluster build() {
+    public Cluster build() {
       // TODO if converting all tests to use builders change ClusterImpl ctor to use list of nodes
       return new ClusterAbstractionsForTest.ClusterImpl(new HashSet<>(buildLiveNodes()), buildClusterCollections());
     }
 
-    List<Node> buildLiveNodes() {
+    public List<Node> buildLiveNodes() {
       List<Node> liveNodes = new LinkedList<>();
       for (NodeBuilder nodeBuilder : nodeBuilders) {
         liveNodes.add(nodeBuilder.build());
@@ -65,7 +65,7 @@ public class Builders {
       return clusterCollections;
     }
 
-    AttributeFetcher buildAttributeFetcher() {
+    public AttributeFetcher buildAttributeFetcher() {
       Map<Node, Integer> nodeToCoreCount = new HashMap<>();
       Map<Node, Long> nodeToFreeDisk = new HashMap<>();
       Map<String, Map<Node, String>> sysprops = new HashMap<>();
@@ -103,17 +103,17 @@ public class Builders {
     }
   }
 
-  static class CollectionBuilder {
+  public static class CollectionBuilder {
     private final String collectionName;
     private LinkedList<ShardBuilder> shardBuilders = new LinkedList<>();
     private Map<String, String> customProperties = new HashMap<>();
 
 
-    private CollectionBuilder(String collectionName) {
+    public CollectionBuilder(String collectionName) {
       this.collectionName = collectionName;
     }
 
-    private CollectionBuilder addCustomProperty(String name, String value) {
+    public CollectionBuilder addCustomProperty(String name, String value) {
       customProperties.put(name, value);
       return this;
     }
@@ -123,7 +123,7 @@ public class Builders {
      * robin to the nodes. The shard leader is the first NRT replica of each shard (or first TLOG is no NRT).
      * Shard and replica configuration can be modified afterwards, the returned builder hierarchy is a convenient starting point.
      */
-    CollectionBuilder initializeShardsReplicas(int countShards, int countNrtReplicas, int countTlogReplicas,
+    public CollectionBuilder initializeShardsReplicas(int countShards, int countNrtReplicas, int countTlogReplicas,
                                                int countPullReplicas, List<NodeBuilder> nodes) {
       Iterator<NodeBuilder> nodeIterator = nodes.iterator();
 
@@ -174,7 +174,7 @@ public class Builders {
       return this;
     }
 
-    SolrCollection build() {
+    public SolrCollection build() {
       ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(collectionName, customProperties);
 
       final LinkedHashMap<String, Shard> shards = new LinkedHashMap<>();
@@ -189,27 +189,27 @@ public class Builders {
     }
   }
 
-  static class ShardBuilder {
+  public static class ShardBuilder {
     private String shardName;
     private LinkedList<ReplicaBuilder> replicaBuilders = new LinkedList<>();
     private ReplicaBuilder leaderReplicaBuilder;
 
-    ShardBuilder setShardName(String shardName) {
+    public ShardBuilder setShardName(String shardName) {
       this.shardName = shardName;
       return this;
     }
 
-    ShardBuilder setReplicaBuilders(LinkedList<ReplicaBuilder> replicaBuilders) {
+    public ShardBuilder setReplicaBuilders(LinkedList<ReplicaBuilder> replicaBuilders) {
       this.replicaBuilders = replicaBuilders;
       return this;
     }
 
-    ShardBuilder setLeader(ReplicaBuilder leaderReplicaBuilder) {
+    public ShardBuilder setLeader(ReplicaBuilder leaderReplicaBuilder) {
       this.leaderReplicaBuilder = leaderReplicaBuilder;
       return this;
     }
 
-    Shard build(SolrCollection collection) {
+    public Shard build(SolrCollection collection) {
       ClusterAbstractionsForTest.ShardImpl shard = new ClusterAbstractionsForTest.ShardImpl(shardName, collection, Shard.ShardState.ACTIVE);
 
       final LinkedHashMap<String, Replica> replicas = new LinkedHashMap<>();
@@ -229,66 +229,66 @@ public class Builders {
     }
   }
 
-  static class ReplicaBuilder {
+  public static class ReplicaBuilder {
     private String replicaName;
     private String coreName;
     private Replica.ReplicaType replicaType;
     private Replica.ReplicaState replicaState;
     private NodeBuilder replicaNode;
 
-    ReplicaBuilder setReplicaName(String replicaName) {
+    public ReplicaBuilder setReplicaName(String replicaName) {
       this.replicaName = replicaName;
       return this;
     }
 
-    ReplicaBuilder setCoreName(String coreName) {
+    public ReplicaBuilder setCoreName(String coreName) {
       this.coreName = coreName;
       return this;
     }
 
-    ReplicaBuilder setReplicaType(Replica.ReplicaType replicaType) {
+    public ReplicaBuilder setReplicaType(Replica.ReplicaType replicaType) {
       this.replicaType = replicaType;
       return this;
     }
 
-    ReplicaBuilder setReplicaState(Replica.ReplicaState replicaState) {
+    public ReplicaBuilder setReplicaState(Replica.ReplicaState replicaState) {
       this.replicaState = replicaState;
       return this;
     }
 
-    ReplicaBuilder setReplicaNode(NodeBuilder replicaNode) {
+    public ReplicaBuilder setReplicaNode(NodeBuilder replicaNode) {
       this.replicaNode = replicaNode;
       return this;
     }
 
-    Replica build(Shard shard) {
+    public Replica build(Shard shard) {
       return new ClusterAbstractionsForTest.ReplicaImpl(replicaName, coreName, shard, replicaType, replicaState, replicaNode.build());
     }
   }
 
-  static class NodeBuilder {
+  public static class NodeBuilder {
     private String nodeName = null;
     private Integer coreCount = null;
     private Long freeDiskGB = null;
     private Map<String, String> sysprops = null;
     private Map<String, Double> metrics = null;
 
-    NodeBuilder setNodeName(String nodeName) {
+    public NodeBuilder setNodeName(String nodeName) {
       this.nodeName = nodeName;
       return this;
     }
 
-    NodeBuilder setCoreCount(Integer coreCount) {
+    public NodeBuilder setCoreCount(Integer coreCount) {
       this.coreCount = coreCount;
       return this;
     }
 
-    NodeBuilder setFreeDiskGB(Long freeDiskGB) {
+    public NodeBuilder setFreeDiskGB(Long freeDiskGB) {
       this.freeDiskGB = freeDiskGB;
       return this;
     }
 
-    NodeBuilder setSysprop(String key, String value) {
+    public NodeBuilder setSysprop(String key, String value) {
       if (sysprops == null) {
         sysprops = new HashMap<>();
       }
@@ -297,7 +297,7 @@ public class Builders {
       return this;
     }
 
-    NodeBuilder setMetric(AttributeFetcher.NodeMetricRegistry registry, String key, Double value) {
+    public NodeBuilder setMetric(AttributeFetcher.NodeMetricRegistry registry, String key, Double value) {
       if (metrics == null) {
         metrics = new HashMap<>();
       }
@@ -306,23 +306,23 @@ public class Builders {
       return this;
     }
 
-    Integer getCoreCount() {
+    public Integer getCoreCount() {
       return coreCount;
     }
 
-    Long getFreeDiskGB() {
+    public Long getFreeDiskGB() {
       return freeDiskGB;
     }
 
-    Map<String, String> getSysprops() {
+    public Map<String, String> getSysprops() {
       return sysprops;
     }
 
-    Map<String, Double> getMetrics() {
+    public Map<String, Double> getMetrics() {
       return metrics;
     }
 
-    Node build() {
+    public Node build() {
       // It is ok to build a new instance each time, that instance does the right thing with equals() and hashCode()
       return new ClusterAbstractionsForTest.NodeImpl(nodeName);
     }
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/ClusterAbstractionsForTest.java
similarity index 99%
rename from solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
rename to solr/core/src/test/org/apache/solr/cluster/placement/ClusterAbstractionsForTest.java
index 059143b..02c1a41 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/ClusterAbstractionsForTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.solr.cluster.placement.impl;
+package org.apache.solr.cluster.placement;
 
 import org.apache.solr.cluster.*;
 
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactoryTest.java
similarity index 96%
rename from solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
rename to solr/core/src/test/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactoryTest.java
index 8f7e130..3869249 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactoryTest.java
@@ -15,16 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.solr.cluster.placement.impl;
+package org.apache.solr.cluster.placement.plugins;
 
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.cluster.Cluster;
 import org.apache.solr.cluster.Node;
 import org.apache.solr.cluster.Replica;
 import org.apache.solr.cluster.Shard;
 import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.*;
-import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
-import org.junit.Assert;
+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;
 import org.slf4j.Logger;
@@ -40,7 +43,7 @@ import java.util.stream.StreamSupport;
 /**
  * Unit test for {@link AffinityPlacementFactory}
  */
-public class AffinityPlacementFactoryTest extends Assert {
+public class AffinityPlacementFactoryTest extends SolrTestCaseJ4 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static PlacementPlugin plugin;
@@ -50,6 +53,7 @@ public class AffinityPlacementFactoryTest extends Assert {
     PlacementPluginConfig config = PlacementPluginConfigImpl.createConfigFromProperties(
         Map.of("minimalFreeDiskGB", 10L, "deprioritizedFreeDiskGB", 50L));
     plugin = new AffinityPlacementFactory().createPluginInstance(config);
+    ((AffinityPlacementFactory.AffinityPlacementPlugin) plugin).setRandom(random());
   }
 
   @Test

[lucene] 29/33: SOLR-15016: Fix more left-overs from hierarchy cleanup.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 0181f85d586a1b82f99fd34bf87e16dc364c7418
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Dec 3 17:52:09 2020 +0100

    SOLR-15016: Fix more left-overs from hierarchy cleanup.
---
 solr/core/src/java/org/apache/solr/core/CoreContainer.java          | 4 ++--
 .../solr/cluster/placement/impl/PlacementPluginIntegrationTest.java | 6 +++---
 2 files changed, 5 insertions(+), 5 deletions(-)

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 346f769..45bffd1 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -74,11 +74,11 @@ 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.PlacementPluginConfig;
 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;
@@ -2188,7 +2188,7 @@ public class CoreContainer {
     return clusterEventProducer;
   }
 
-  public PlacementPluginFactory<? extends MapWriter> getPlacementPluginFactory() {
+  public PlacementPluginFactory<? extends PlacementPluginConfig> getPlacementPluginFactory() {
     return placementPluginFactory;
   }
 
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 f17ef00..676c70f 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
@@ -24,12 +24,12 @@ import org.apache.solr.client.solrj.request.beans.PluginMeta;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.V2Response;
 import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.cluster.placement.PlacementPluginConfig;
 import org.apache.solr.cluster.placement.PlacementPluginFactory;
 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.MapWriter;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.util.TimeSource;
@@ -135,7 +135,7 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
   @Test
   @SuppressWarnings("unchecked")
   public void testDynamicReconfiguration() throws Exception {
-    PlacementPluginFactory<? extends MapWriter> pluginFactory = cc.getPlacementPluginFactory();
+    PlacementPluginFactory<? extends PlacementPluginConfig> pluginFactory = cc.getPlacementPluginFactory();
     assertTrue("wrong type " + pluginFactory.getClass().getName(), pluginFactory instanceof DelegatingPlacementPluginFactory);
     DelegatingPlacementPluginFactory wrapper = (DelegatingPlacementPluginFactory) pluginFactory;
 
@@ -155,7 +155,7 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
     version = waitForVersionChange(version, wrapper, 10);
 
     assertTrue("wrong version " + version, version > 0);
-    PlacementPluginFactory<? extends MapWriter> factory = wrapper.getDelegate();
+    PlacementPluginFactory<? extends PlacementPluginConfig> factory = wrapper.getDelegate();
     assertTrue("wrong type " + factory.getClass().getName(), factory instanceof MinimizeCoresPlacementFactory);
 
     // reconfigure

[lucene] 05/33: Inverted the plugin and factory class hierarchy for each sample plugin

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit be681a3afaaacbed0630e6b75f1f2414e25ca3b8
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Thu Nov 19 15:48:18 2020 +0100

    Inverted the plugin and factory class hierarchy for each sample plugin
---
 .../placement/impl/PlacementPluginConfigImpl.java  |   4 +-
 .../plugins/AffinityPlacementFactory.java          | 511 +++++++++++++++++++++
 .../plugins/MinimizeCoresPlacementFactory.java     | 128 ++++++
 .../placement/plugins/RandomPlacementFactory.java  |  86 ++++
 .../SamplePluginAffinityReplicaPlacement.java      | 509 --------------------
 .../plugins/SamplePluginMinimizeCores.java         | 138 ------
 .../plugins/SamplePluginRandomPlacement.java       |  88 ----
 ...Test.java => AffinityPlacementFactoryTest.java} |   8 +-
 .../impl/PlacementPluginIntegrationTest.java       |   4 +-
 9 files changed, 733 insertions(+), 743 deletions(-)

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
index b21cbfd..4c079d4 100644
--- 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
@@ -24,7 +24,7 @@ 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.SamplePluginAffinityReplicaPlacement;
+import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.Utils;
 
@@ -110,7 +110,7 @@ public class PlacementPluginConfigImpl implements PlacementPluginConfig {
    * <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 SamplePluginAffinityReplicaPlacement}.</p>
+   * <p>See configuration example and how-to in {@link AffinityPlacementFactory}.</p>
    */
   static PlacementPluginConfig createConfigFromProperties(Map<String, Object> pluginConfig) {
     final Map<String, String> stringConfigs = new HashMap<>();
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
new file mode 100644
index 0000000..db968e8
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java
@@ -0,0 +1,511 @@
+/*
+ * 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.plugins;
+
+import com.google.common.collect.*;
+import org.apache.solr.cluster.*;
+import org.apache.solr.cluster.placement.*;
+import org.apache.solr.common.util.Pair;
+import org.apache.solr.common.util.SuppressForbidden;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.invoke.MethodHandles;
+import java.util.*;
+import java.util.stream.Collectors;
+
+/**
+ * <p>This factory is instantiated by config from its class name. Using it is the only way to create instances of
+ * {@link AffinityPlacementPlugin}.</p>
+ *
+ * <p>In order to configure this plugin to be used for placement decisions, the following {@code curl} command (or something
+ * equivalent) has to be executed once the cluster is already running in order to set
+ * the appropriate Zookeeper stored configuration. Replace {@code localhost:8983} by one of your servers' IP address and port.</p>
+ *
+ * <pre>
+ *
+  curl -X POST -H 'Content-type:application/json' -d '{
+    "set-placement-plugin": {
+      "class": "org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory",
+      "minimalFreeDiskGB": 10,
+      "deprioritizedFreeDiskGB": 50
+    }
+  }' http://localhost:8983/api/cluster
+ * </pre>
+ *
+ * <p>The consequence will be the creation of an element in the Zookeeper file {@code /clusterprops.json} as follows:</p>
+ *
+ * <pre>
+ *
+ * "placement-plugin":{
+ *     "class":"org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory",
+ *     "minimalFreeDiskGB":10,
+ *     "deprioritizedFreeDiskGB":50}
+ * </pre>
+ *
+ * <p>In order to delete the placement-plugin section from {@code /clusterprops.json} (and to fallback to either Legacy
+ * or rule based placement if configured for a collection), execute:</p>
+ *
+ * <pre>
+ *
+  curl -X POST -H 'Content-type:application/json' -d '{
+    "set-placement-plugin" : null
+  }' http://localhost:8983/api/cluster
+ * </pre>
+ */
+public class AffinityPlacementFactory implements PlacementPluginFactory {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * 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
+   * having to call a constructor with more parameters (if we were to instantiate the plugin class directly without going
+   * through a factory).
+   */
+  public AffinityPlacementFactory() {
+  }
+
+  @Override
+  public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
+    final long minimalFreeDiskGB = config.getLongConfig("minimalFreeDiskGB", 20L);
+    final long deprioritizedFreeDiskGB = config.getLongConfig("deprioritizedFreeDiskGB", 100L);
+    return new AffinityPlacementPlugin(minimalFreeDiskGB, deprioritizedFreeDiskGB);
+  }
+
+  /**
+   * <p>Implements placing replicas in a way that replicate past Autoscaling config defined
+   * <a href="https://github.com/lucidworks/fusion-cloud-native/blob/master/policy.json#L16">here</a>.</p>
+   *
+   * <p>This specification is doing the following:
+   * <p><i>Spread replicas per shard as evenly as possible across multiple availability zones (given by a sys prop),
+   * assign replicas based on replica type to specific kinds of nodes (another sys prop), and avoid having more than
+   * one replica per shard on the same node.<br>
+   * Only after these constraints are satisfied do minimize cores per node or disk usage.</i></p>
+   *
+   * <p>Overall strategy of this plugin:</p>
+   * <ul><li>
+   *     The set of nodes in the cluster is obtained and transformed into 3 independent sets (that can overlap) of nodes
+   *     accepting each of the three replica types.
+   * </li><li>
+   *     For each shard on which placing replicas is required and then for each replica type to place (starting with NRT,
+   *     then TLOG then PULL): <ul>
+   *         <li>The set of candidates nodes corresponding to the replica type is used and from that set are removed nodes
+   *         that already have a replica (of any type) for that shard</li>
+   *         <li>If there are not enough nodes, an error is thrown (this is checked further down during processing).</li>
+   *         <li>The number of (already existing) replicas of the current type on each Availability Zone is collected.</li>
+   *         <li>Separate the set of available nodes to as many subsets (possibly some are empty) as there are Availability Zones
+   *         defined for the candidate nodes</li>
+   *         <li>In each AZ nodes subset, sort the nodes by increasing total number of cores count, with possibly a condition
+   *         that pushes nodes with low disk space to the end of the list? Or a weighted combination of the relative
+   *         importance of these two factors? Some randomization? Marking as non available nodes with not enough disk space?
+   *         These and other are likely aspects to be played with once the plugin is tested or observed to be running in prod,
+   *         don't expect the initial code drop(s) to do all of that.</li>
+   *         <li>Iterate over the number of replicas to place (for the current replica type for the current shard):
+   *         <ul>
+   *             <li>Based on the number of replicas per AZ collected previously, pick the non empty set of nodes having the
+   *             lowest number of replicas. Then pick the first node in that set. That's the node the replica is placed one.
+   *             Remove the node from the set of available nodes for the given AZ and increase the number of replicas placed
+   *             on that AZ.</li>
+   *         </ul></li>
+   *         <li>During this process, the number of cores on the nodes in general is tracked to take into account placement
+   *         decisions so that not all shards decide to put their replicas on the same nodes (they might though if these are
+   *         the less loaded nodes).</li>
+   *     </ul>
+   * </li>
+   * </ul>
+   *
+   * <p>This code is a realistic placement computation, based on a few assumptions. The code is written in such a way to
+   * 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>
+   *
+   * See {@link AffinityPlacementFactory} for instructions on how to configure a cluster to use this plugin.
+   */
+  static private class AffinityPlacementPlugin implements PlacementPlugin {
+    /**
+     * <p>Name of the system property on a node indicating which (public cloud) Availability Zone that node is in. The value
+     * is any string, different strings denote different availability zones.
+     *
+     * <p>Nodes on which this system property is not defined are considered being in the same Availability Zone
+     * {@link #UNDEFINED_AVAILABILITY_ZONE} (hopefully the value of this constant is not the name of a real Availability Zone :).
+     */
+    public static final String AVAILABILITY_ZONE_SYSPROP = "availability_zone";
+    /** This is the "AZ" name for nodes that do not define an AZ. Should not match a real AZ name (I think we're safe) */
+    public static final String UNDEFINED_AVAILABILITY_ZONE = "uNd3f1NeD";
+
+    /**
+     * <p>Name of the system property on a node indicating the type of replicas allowed on that node.
+     * The value of that system property is a comma separated list or a single string of value names of
+     * {@link org.apache.solr.cluster.Replica.ReplicaType} (case insensitive). If that property is not defined, that node is
+     * considered accepting all replica types (i.e. undefined is equivalent to {@code "NRT,Pull,tlog"}).
+     *
+     * <p>See {@link #getNodesPerReplicaType}.
+     */
+    public static final String REPLICA_TYPE_SYSPROP = "replica_type";
+
+    /**
+     * 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.
+     */
+    private final 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).
+     */
+    private final long deprioritizedFreeDiskGB;
+
+    /**
+     * The factory has decoded the configuration for the plugin instance and passes it the parameters it needs.
+     */
+    private AffinityPlacementPlugin(long minimalFreeDiskGB, long deprioritizedFreeDiskGB) {
+      this.minimalFreeDiskGB = minimalFreeDiskGB;
+      this.deprioritizedFreeDiskGB = deprioritizedFreeDiskGB;
+    }
+
+    @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
+    public PlacementPlan computePlacement(Cluster cluster, PlacementRequest request, AttributeFetcher attributeFetcher,
+                                          PlacementPlanFactory placementPlanFactory) throws PlacementException {
+      Set<Node> nodes = request.getTargetNodes();
+      SolrCollection solrCollection = request.getCollection();
+
+      // Request all needed attributes
+      attributeFetcher.requestNodeSystemProperty(AVAILABILITY_ZONE_SYSPROP).requestNodeSystemProperty(REPLICA_TYPE_SYSPROP);
+      attributeFetcher.requestNodeCoreCount().requestNodeFreeDisk();
+      attributeFetcher.fetchFrom(nodes);
+      final AttributeValues attrValues = attributeFetcher.fetchAttributes();
+
+      // Split the set of nodes into 3 sets of nodes accepting each replica type (sets can overlap if nodes accept multiple replica types)
+      // These subsets sets are actually maps, because we capture the number of cores (of any replica type) present on each node.
+      // Also get the number of currently existing cores per node, so we can keep update as we place new cores to not end up
+      // always selecting the same node(s).
+      Pair<EnumMap<Replica.ReplicaType, Set<Node>>, Map<Node, Integer>> p = getNodesPerReplicaType(nodes, attrValues);
+
+      EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes = p.first();
+      Map<Node, Integer> coresOnNodes = p.second();
+
+      // All available zones of live nodes. Due to some nodes not being candidates for placement, and some existing replicas
+      // being one availability zones that might be offline (i.e. their nodes are not live), this set might contain zones
+      // on which it is impossible to place replicas. That's ok.
+      ImmutableSet<String> availabilityZones = getZonesFromNodes(nodes, attrValues);
+
+      // Build the replica placement decisions here
+      Set<ReplicaPlacement> replicaPlacements = new HashSet<>();
+
+      // Let's now iterate on all shards to create replicas for and start finding home sweet homes for the replicas
+      for (String shardName : request.getShardNames()) {
+        // Iterate on the replica types in the enum order. We place more strategic replicas first
+        // (NRT is more strategic than TLOG more strategic than PULL). This is in case we eventually decide that less
+        // strategic replica placement impossibility is not a problem that should lead to replica placement computation
+        // failure. Current code does fail if placement is impossible (constraint is at most one replica of a shard on any node).
+        for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
+          makePlacementDecisions(solrCollection, shardName, availabilityZones, replicaType, request.getCountReplicasToCreate(replicaType),
+                  attrValues, replicaTypeToNodes, coresOnNodes, placementPlanFactory, replicaPlacements);
+        }
+      }
+
+      return placementPlanFactory.createPlacementPlan(request, replicaPlacements);
+    }
+
+    private ImmutableSet<String> getZonesFromNodes(Set<Node> nodes, final AttributeValues attrValues) {
+      Set<String> azs = new HashSet<>();
+
+      for (Node n : nodes) {
+        azs.add(getNodeAZ(n, attrValues));
+      }
+
+      return ImmutableSet.copyOf(azs);
+    }
+
+    /**
+     * Resolves the AZ of a node and takes care of nodes that have no defined AZ in system property {@link #AVAILABILITY_ZONE_SYSPROP}
+     * to then return {@link #UNDEFINED_AVAILABILITY_ZONE} as the AZ name.
+     */
+    private String getNodeAZ(Node n, final AttributeValues attrValues) {
+      Optional<String> nodeAz = attrValues.getSystemProperty(n, AVAILABILITY_ZONE_SYSPROP);
+      // All nodes with undefined AZ will be considered part of the same AZ. This also works for deployments that do not care about AZ's
+      return nodeAz.orElse(UNDEFINED_AVAILABILITY_ZONE);
+    }
+
+    /**
+     * This class captures an availability zone and the nodes that are legitimate targets for replica placement in that
+     * Availability Zone. Instances are used as values in a {@link TreeMap} in which the total number of already
+     * existing replicas in the AZ is the key. This allows easily picking the set of nodes from which to select a node for
+     * placement in order to balance the number of replicas per AZ. Picking one of the nodes from the set is done using
+     * different criteria unrelated to the Availability Zone (picking the node is based on the {@link CoresAndDiskComparator}
+     * ordering).
+     */
+    private static class AzWithNodes {
+      final String azName;
+      List<Node> availableNodesForPlacement;
+      boolean hasBeenSorted;
+
+      AzWithNodes(String azName, List<Node> availableNodesForPlacement) {
+        this.azName = azName;
+        this.availableNodesForPlacement = availableNodesForPlacement;
+        // Once the list is sorted to an order we're happy with, this flag is set to true to avoid sorting multiple times
+        // unnecessarily.
+        this.hasBeenSorted = false;
+      }
+    }
+
+    /**
+     * Given the set of all nodes on which to do placement and fetched attributes, builds the sets representing
+     * candidate nodes for placement of replicas of each replica type.
+     * These sets are packaged and returned in an EnumMap keyed by replica type (1st member of the Pair).
+     * Also builds the number of existing cores on each node present in the returned EnumMap (2nd member of the returned Pair).
+     * Nodes for which the number of cores is not available for whatever reason are excluded from acceptable candidate nodes
+     * as it would not be possible to make any meaningful placement decisions.
+     * @param nodes all nodes on which this plugin should compute placement
+     * @param attrValues attributes fetched for the nodes. This method uses system property {@link #REPLICA_TYPE_SYSPROP} as
+     *                   well as the number of cores on each node.
+     */
+    private Pair<EnumMap<Replica.ReplicaType, Set<Node>>, Map<Node, Integer>> getNodesPerReplicaType(Set<Node> nodes, final AttributeValues attrValues) {
+      EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes = new EnumMap<>(Replica.ReplicaType.class);
+      Map<Node, Integer> coresOnNodes = Maps.newHashMap();
+
+      for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
+        replicaTypeToNodes.put(replicaType, new HashSet<>());
+      }
+
+      for (Node node : nodes) {
+        // Exclude nodes with unknown or too small disk free space
+        if (attrValues.getFreeDisk(node).isEmpty()) {
+          if (log.isWarnEnabled()) {
+            log.warn("Unknown free disk on node {}, excluding it from placement decisions.", node.getName());
+          }
+          // We rely later on the fact that the free disk optional is present (see CoresAndDiskComparator), be careful it you change anything here.
+          continue;
+        } if (attrValues.getFreeDisk(node).get() < minimalFreeDiskGB) {
+          if (log.isWarnEnabled()) {
+            log.warn("Node {} free disk ({}GB) lower than configured minimum {}GB, excluding it from placement decisions.", node.getName(), attrValues.getFreeDisk(node).get(), minimalFreeDiskGB);
+          }
+          continue;
+        }
+
+        if (attrValues.getCoresCount(node).isEmpty()) {
+          if (log.isWarnEnabled()) {
+            log.warn("Unknown number of cores on node {}, excluding it from placement decisions.", node.getName());
+          }
+          // We rely later on the fact that the number of cores optional is present (see CoresAndDiskComparator), be careful it you change anything here.
+          continue;
+        }
+
+        Integer coresCount = attrValues.getCoresCount(node).get();
+        coresOnNodes.put(node, coresCount);
+
+        String supportedReplicaTypes = attrValues.getSystemProperty(node, REPLICA_TYPE_SYSPROP).isPresent() ? attrValues.getSystemProperty(node, REPLICA_TYPE_SYSPROP).get() : null;
+        // If property not defined or is only whitespace on a node, assuming node can take any replica type
+        if (supportedReplicaTypes == null || supportedReplicaTypes.isBlank()) {
+          for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
+            replicaTypeToNodes.get(rt).add(node);
+          }
+        } else {
+          Set<String> acceptedTypes = Arrays.stream(supportedReplicaTypes.split(",")).map(String::trim).map(s -> s.toLowerCase(Locale.ROOT)).collect(Collectors.toSet());
+          for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
+            if (acceptedTypes.contains(rt.name().toLowerCase(Locale.ROOT))) {
+              replicaTypeToNodes.get(rt).add(node);
+            }
+          }
+        }
+      }
+      return new Pair<>(replicaTypeToNodes, coresOnNodes);
+    }
+
+    /**
+     * <p>Picks nodes from {@code targetNodes} for placing {@code numReplicas} replicas.
+     *
+     * <p>The criteria used in this method are, in this order:
+     * <ol>
+     *     <li>No more than one replica of a given shard on a given node (strictly enforced)</li>
+     *     <li>Balance as much as possible the number of replicas of the given {@link org.apache.solr.cluster.Replica.ReplicaType} over available AZ's.
+     *     This balancing takes into account existing replicas <b>of the corresponding replica type</b>, if any.</li>
+     *     <li>Place replicas is possible on nodes having more than a certain amount of free disk space (note that nodes with a too small
+     *     amount of free disk space were eliminated as placement targets earlier, in {@link #getNodesPerReplicaType}). There's
+     *     a threshold here rather than sorting on the amount of free disk space, because sorting on that value would in
+     *     practice lead to never considering the number of cores on a node.</li>
+     *     <li>Place replicas on nodes having a smaller number of cores (the number of cores considered
+     *     for this decision includes decisions made during the processing of the placement request)</li>
+     * </ol>
+     */
+    @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
+    private void makePlacementDecisions(SolrCollection solrCollection, String shardName, ImmutableSet<String> availabilityZones,
+                                        Replica.ReplicaType replicaType, int numReplicas, final AttributeValues attrValues,
+                                        EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes, Map<Node, Integer> coresOnNodes,
+                                        PlacementPlanFactory placementPlanFactory, Set<ReplicaPlacement> replicaPlacements) throws PlacementException {
+      // Build the set of candidate nodes, i.e. nodes not having (yet) a replica of the given shard
+      Set<Node> candidateNodes = new HashSet<>(replicaTypeToNodes.get(replicaType));
+
+      // Count existing replicas per AZ. We count only instances the type of replica for which we need to do placement. This
+      // can be changed in the loop below if we want to count all replicas for the shard.
+      Map<String, Integer> azToNumReplicas = Maps.newHashMap();
+      // Add all "interesting" AZ's, i.e. AZ's for which there's a chance we can do placement.
+      for (String az : availabilityZones) {
+        azToNumReplicas.put(az, 0);
+      }
+
+      Shard shard = solrCollection.getShard(shardName);
+      if (shard != null) {
+        // shard is non null if we're adding replicas to an already existing collection.
+        // If we're creating the collection, the shards do not exist yet.
+        for (Replica replica : shard.replicas()) {
+          // Nodes already having any type of replica for the shard can't get another replica.
+          candidateNodes.remove(replica.getNode());
+          // The node's AZ has to be counted as having a replica if it has a replica of the same type as the one we need
+          // to place here (remove the "if" below to balance the number of replicas per AZ across all replica types rather
+          // than within each replica type, but then there's a risk that all NRT replicas for example end up on the same AZ).
+          // Note that if in the cluster nodes are configured to accept a single replica type and not multiple ones, the
+          // two options are equivalent (governed by system property AVAILABILITY_ZONE_SYSPROP on each node)
+          if (replica.getType() == replicaType) {
+            final String az = getNodeAZ(replica.getNode(), attrValues);
+            if (azToNumReplicas.containsKey(az)) {
+              // We do not count replicas on AZ's for which we don't have any node to place on because it would not help
+              // the placement decision. If we did want to do that, note the dereferencing below can't be assumed as the
+              // entry will not exist in the map.
+              azToNumReplicas.put(az, azToNumReplicas.get(az) + 1);
+            }
+          }
+        }
+      }
+
+      // We now have the set of real candidate nodes, we've enforced "No more than one replica of a given shard on a given node".
+      // We also counted for the shard and replica type under consideration how many replicas were per AZ, so we can place
+      // (or try to place) replicas on AZ's that have fewer replicas
+
+      // Get the candidate nodes per AZ in order to build (further down) a mapping of AZ to placement candidates.
+      Map<String, List<Node>> nodesPerAz = Maps.newHashMap();
+      for (Node node : candidateNodes) {
+        String nodeAz = getNodeAZ(node, attrValues);
+        List<Node> nodesForAz = nodesPerAz.computeIfAbsent(nodeAz, k -> new ArrayList<>());
+        nodesForAz.add(node);
+      }
+
+      // Build a treeMap sorted by the number of replicas per AZ and including candidates nodes suitable for placement on the
+      // AZ, so we can easily select the next AZ to get a replica assignment and quickly (constant time) decide if placement
+      // on this AZ is possible or not.
+      TreeMultimap<Integer, AzWithNodes> azByExistingReplicas = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());
+      for (Map.Entry<String, List<Node>> e : nodesPerAz.entrySet()) {
+        azByExistingReplicas.put(azToNumReplicas.get(e.getKey()), new AzWithNodes(e.getKey(), e.getValue()));
+      }
+
+      CoresAndDiskComparator coresAndDiskComparator = new CoresAndDiskComparator(attrValues, coresOnNodes, deprioritizedFreeDiskGB);
+
+      // Now we have for each AZ on which we might have a chance of placing a replica, the list of candidate nodes for replicas
+      // (candidate: does not already have a replica of this shard and is in the corresponding AZ).
+      // We must now select those of the nodes on which we actually place the replicas, and will do that based on the total
+      // number of cores already present on these nodes as well as the free disk space.
+      // We sort once by the order related to number of cores and disk space each list of nodes on an AZ. We do not sort all
+      // of them ahead of time because we might be placing a small number of replicas and it might be wasted work.
+      for (int i = 0; i < numReplicas; i++) {
+        // Pick the AZ having the lowest number of replicas for this shard, and if that AZ has available nodes, pick the
+        // most appropriate one (based on number of cores and disk space constraints). In the process, remove entries (AZ's)
+        // that do not have nodes to place replicas on because these are useless to us.
+        Map.Entry<Integer, AzWithNodes> azWithNodesEntry = null;
+        for (Iterator<Map.Entry<Integer, AzWithNodes>> it = azByExistingReplicas.entries().iterator(); it.hasNext(); ) {
+          Map.Entry<Integer, AzWithNodes> entry = it.next();
+          if (!entry.getValue().availableNodesForPlacement.isEmpty()) {
+            azWithNodesEntry = entry;
+            // Remove this entry. Will add it back after a node has been removed from the list of available nodes and the number
+            // of replicas on the AZ has been increased by one (search for "azByExistingReplicas.put" below).
+            it.remove();
+            break;
+          } else {
+            it.remove();
+          }
+        }
+
+        if (azWithNodesEntry == null) {
+          // This can happen because not enough nodes for the placement request or already too many nodes with replicas of
+          // the shard that can't accept new replicas or not enough nodes with enough free disk space.
+          throw new PlacementException("Not enough nodes to place " + numReplicas + " replica(s) of type " + replicaType +
+                  " for shard " + shardName + " of collection " + solrCollection.getName());
+        }
+
+        AzWithNodes azWithNodes = azWithNodesEntry.getValue();
+        List<Node> nodes = azWithNodes.availableNodesForPlacement;
+
+        if (!azWithNodes.hasBeenSorted) {
+          // Make sure we do not tend to use always the same nodes (within an AZ) if all conditions are identical (well, this
+          // likely is not the case since after having added a replica to a node its number of cores increases for the next
+          // placement decision, but let's be defensive here, given that multiple concurrent placement decisions might see
+          // the same initial cluster state, and we want placement to be reasonable even in that case without creating an
+          // unnecessary imbalance).
+          // For example, if all nodes have 0 cores and same amount of free disk space, ideally we want to pick a random node
+          // for placement, not always the same one due to some internal ordering.
+          Collections.shuffle(nodes, new Random());
+
+          // Sort by increasing number of cores but pushing nodes with low free disk space to the end of the list
+          nodes.sort(coresAndDiskComparator);
+
+          azWithNodes.hasBeenSorted = true;
+        }
+
+        Node assignTarget = nodes.remove(0);
+
+        // Insert back a corrected entry for the AZ: one more replica living there and one less node that can accept new replicas
+        // (the remaining candidate node list might be empty, in which case it will be cleaned up on the next iteration).
+        azByExistingReplicas.put(azWithNodesEntry.getKey() + 1, azWithNodes);
+
+        // Track that the node has one more core. These values are only used during the current run of the plugin.
+        coresOnNodes.merge(assignTarget, 1, Integer::sum);
+
+        // Register the replica assignment just decided
+        replicaPlacements.add(placementPlanFactory.createReplicaPlacement(solrCollection, shardName, assignTarget, replicaType));
+      }
+    }
+
+    /**
+     * Comparator implementing the placement strategy based on free space and number of cores: we want to place new replicas
+     * on nodes with the less number of cores, but only if they do have enough disk space (expressed as a threshold value).
+     */
+    static class CoresAndDiskComparator implements Comparator<Node> {
+      private final AttributeValues attrValues;
+      private final Map<Node, Integer> coresOnNodes;
+      private final long deprioritizedFreeDiskGB;
+
+
+      /**
+       * The data we sort on is not part of the {@link Node} instances but has to be retrieved from the attributes and configuration.
+       * The number of cores per node is passed in a map whereas the free disk is fetched from the attributes due to the
+       * fact that we update the number of cores per node as we do allocations, but we do not update the free disk. The
+       * attrValues correpsonding to the number of cores per node are the initial values, but we want to comapre the actual
+       * value taking into account placement decisions already made during the current execution of the placement plugin.
+       */
+      CoresAndDiskComparator(AttributeValues attrValues, Map<Node, Integer> coresOnNodes, long deprioritizedFreeDiskGB) {
+        this.attrValues = attrValues;
+        this.coresOnNodes = coresOnNodes;
+        this.deprioritizedFreeDiskGB = deprioritizedFreeDiskGB;
+      }
+
+      @Override
+      public int compare(Node a, Node b) {
+        // Note all nodes do have free disk defined. This has been verified earlier.
+        boolean aHasLowFreeSpace = attrValues.getFreeDisk(a).get() < deprioritizedFreeDiskGB;
+        boolean bHasLowFreeSpace = attrValues.getFreeDisk(b).get() < deprioritizedFreeDiskGB;
+        if (aHasLowFreeSpace != bHasLowFreeSpace) {
+          // A node with low free space should be considered > node with high free space since it needs to come later in sort order
+          return Boolean.compare(aHasLowFreeSpace, bHasLowFreeSpace);
+        }
+        // The ordering on the number of cores is the natural order.
+        return Integer.compare(coresOnNodes.get(a), coresOnNodes.get(b));
+      }
+    }
+  }
+}
+
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
new file mode 100644
index 0000000..33d9c57
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/MinimizeCoresPlacementFactory.java
@@ -0,0 +1,128 @@
+/*
+ * 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.plugins;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.Map;
+
+import com.google.common.collect.Ordering;
+import com.google.common.collect.TreeMultimap;
+import org.apache.solr.cluster.Cluster;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.SolrCollection;
+import org.apache.solr.cluster.placement.*;
+import org.apache.solr.common.util.SuppressForbidden;
+
+/**
+ * <p>Factory for creating {@link MinimizeCoresPlacementPlugin}.</p>
+ *
+ * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
+ */
+public class MinimizeCoresPlacementFactory implements PlacementPluginFactory {
+
+  @Override
+  public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
+    return new MinimizeCoresPlacementPlugin();
+  }
+
+  /**
+   * Placement plugin implementing placing replicas to minimize number of cores per {@link Node}, while not placing two
+   * replicas of the same shard on the same node. This code is meant as an educational example of a placement plugin.
+   */
+  static private class MinimizeCoresPlacementPlugin implements PlacementPlugin {
+
+    @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
+    public PlacementPlan computePlacement(Cluster cluster, PlacementRequest request, AttributeFetcher attributeFetcher,
+                                          PlacementPlanFactory placementPlanFactory) throws PlacementException {
+      int totalReplicasPerShard = 0;
+      for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
+        totalReplicasPerShard += request.getCountReplicasToCreate(rt);
+      }
+
+      if (cluster.getLiveNodes().size() < totalReplicasPerShard) {
+        throw new PlacementException("Cluster size too small for number of replicas per shard");
+      }
+
+      // Get number of cores on each Node
+      TreeMultimap<Integer, Node> nodesByCores = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());
+
+      Set<Node> nodes = request.getTargetNodes();
+
+      attributeFetcher.requestNodeCoreCount();
+      attributeFetcher.fetchFrom(nodes);
+      AttributeValues attrValues = attributeFetcher.fetchAttributes();
+
+
+      // Get the number of cores on each node and sort the nodes by increasing number of cores
+      for (Node node : nodes) {
+        if (attrValues.getCoresCount(node).isEmpty()) {
+          throw new PlacementException("Can't get number of cores in " + node);
+        }
+        nodesByCores.put(attrValues.getCoresCount(node).get(), node);
+      }
+
+      Set<ReplicaPlacement> replicaPlacements = new HashSet<>(totalReplicasPerShard * request.getShardNames().size());
+
+      // Now place all replicas of all shards on nodes, by placing on nodes with the smallest number of cores and taking
+      // into account replicas placed during this computation. Note that for each shard we must place replicas on different
+      // nodes, when moving to the next shard we use the nodes sorted by their updated number of cores (due to replica
+      // placements for previous shards).
+      for (String shardName : request.getShardNames()) {
+        // Assign replicas based on the sort order of the nodesByCores tree multimap to put replicas on nodes with less
+        // cores first. We only need totalReplicasPerShard nodes given that's the number of replicas to place.
+        // We assign based on the passed nodeEntriesToAssign list so the right nodes get replicas.
+        ArrayList<Map.Entry<Integer, Node>> nodeEntriesToAssign = new ArrayList<>(totalReplicasPerShard);
+        Iterator<Map.Entry<Integer, Node>> treeIterator = nodesByCores.entries().iterator();
+        for (int i = 0; i < totalReplicasPerShard; i++) {
+          nodeEntriesToAssign.add(treeIterator.next());
+        }
+
+        // Update the number of cores each node will have once the assignments below got executed so the next shard picks the
+        // lowest loaded nodes for its replicas.
+        for (Map.Entry<Integer, Node> e : nodeEntriesToAssign) {
+          int coreCount = e.getKey();
+          Node node = e.getValue();
+          nodesByCores.remove(coreCount, node);
+          nodesByCores.put(coreCount + 1, node);
+        }
+
+        for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
+          placeReplicas(request.getCollection(), nodeEntriesToAssign, placementPlanFactory, replicaPlacements, shardName, request, replicaType);
+        }
+      }
+
+      return placementPlanFactory.createPlacementPlan(request, replicaPlacements);
+    }
+
+    private void placeReplicas(SolrCollection solrCollection, ArrayList<Map.Entry<Integer, Node>> nodeEntriesToAssign,
+                               PlacementPlanFactory placementPlanFactory, Set<ReplicaPlacement> replicaPlacements,
+                               String shardName, PlacementRequest request, Replica.ReplicaType replicaType) {
+      for (int replica = 0; replica < request.getCountReplicasToCreate(replicaType); replica++) {
+        final Map.Entry<Integer, Node> entry = nodeEntriesToAssign.remove(0);
+        final Node node = entry.getValue();
+
+        replicaPlacements.add(placementPlanFactory.createReplicaPlacement(solrCollection, shardName, node, replicaType));
+      }
+    }
+  }
+}
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
new file mode 100644
index 0000000..7874c33
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/RandomPlacementFactory.java
@@ -0,0 +1,86 @@
+/*
+ * 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.plugins;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.solr.cluster.Cluster;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.SolrCollection;
+import org.apache.solr.cluster.placement.*;
+
+/**
+ * <p>Factory for creating {@link RandomPlacementPlugin}.</p>
+ *
+ * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
+ */
+public class RandomPlacementFactory implements PlacementPluginFactory {
+
+  @Override
+  public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
+    return new RandomPlacementPlugin();
+  }
+
+  /**
+   * Implements random placement for new collection creation while preventing two replicas of same shard from being placed on same node.
+   */
+  static private class RandomPlacementPlugin implements PlacementPlugin {
+
+    public PlacementPlan computePlacement(Cluster cluster, PlacementRequest request, AttributeFetcher attributeFetcher,
+                                          PlacementPlanFactory placementPlanFactory) throws PlacementException {
+      int totalReplicasPerShard = 0;
+      for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
+        totalReplicasPerShard += request.getCountReplicasToCreate(rt);
+      }
+
+      if (cluster.getLiveNodes().size() < totalReplicasPerShard) {
+        throw new PlacementException("Cluster size too small for number of replicas per shard");
+      }
+
+      Set<ReplicaPlacement> replicaPlacements = new HashSet<>(totalReplicasPerShard * request.getShardNames().size());
+
+      // Now place randomly all replicas of all shards on available nodes
+      for (String shardName : request.getShardNames()) {
+        // Shuffle the nodes for each shard so that replicas for a shard are placed on distinct yet random nodes
+        ArrayList<Node> nodesToAssign = new ArrayList<>(cluster.getLiveNodes());
+        Collections.shuffle(nodesToAssign, new Random());
+
+        for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
+          placeForReplicaType(request.getCollection(), nodesToAssign, placementPlanFactory, replicaPlacements, shardName, request, rt);
+        }
+      }
+
+      return placementPlanFactory.createPlacementPlan(request, replicaPlacements);
+    }
+
+    private void placeForReplicaType(SolrCollection solrCollection, ArrayList<Node> nodesToAssign, PlacementPlanFactory placementPlanFactory,
+                                     Set<ReplicaPlacement> replicaPlacements,
+                                     String shardName, PlacementRequest request, Replica.ReplicaType replicaType) {
+      for (int replica = 0; replica < request.getCountReplicasToCreate(replicaType); replica++) {
+        Node node = nodesToAssign.remove(0);
+
+        replicaPlacements.add(placementPlanFactory.createReplicaPlacement(solrCollection, shardName, node, replicaType));
+      }
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginAffinityReplicaPlacement.java b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginAffinityReplicaPlacement.java
deleted file mode 100644
index d738fb8..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginAffinityReplicaPlacement.java
+++ /dev/null
@@ -1,509 +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.plugins;
-
-import com.google.common.collect.*;
-import org.apache.solr.cluster.*;
-import org.apache.solr.cluster.placement.*;
-import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.SuppressForbidden;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.invoke.MethodHandles;
-import java.util.*;
-import java.util.stream.Collectors;
-
-/**
- * <p>Implements placing replicas in a way that replicate past Autoscaling config defined
- * <a href="https://github.com/lucidworks/fusion-cloud-native/blob/master/policy.json#L16">here</a>.</p>
- *
- * <p>This specification is doing the following:
- * <p><i>Spread replicas per shard as evenly as possible across multiple availability zones (given by a sys prop),
- * assign replicas based on replica type to specific kinds of nodes (another sys prop), and avoid having more than
- * one replica per shard on the same node.<br>
- * Only after these constraints are satisfied do minimize cores per node or disk usage.</i></p>
- *
- * <p>Overall strategy of this plugin:</p>
- * <ul><li>
- *     The set of nodes in the cluster is obtained and transformed into 3 independent sets (that can overlap) of nodes
- *     accepting each of the three replica types.
- * </li><li>
- *     For each shard on which placing replicas is required and then for each replica type to place (starting with NRT, then TLOG then PULL): <ul>
- *         <li>The set of candidates nodes corresponding to the replica type is used and from that set are removed nodes
- *         that already have a replica (of any type) for that shard</li>
- *         <li>If there are not enough nodes, an error is thrown (this is checked further down during processing).</li>
- *         <li>The number of (already existing) replicas of the current type on each Availability Zone is collected.</li>
- *         <li>Separate the set of available nodes to as many subsets (possibly some are empty) as there are Availability Zones
- *         defined for the candidate nodes</li>
- *         <li>In each AZ nodes subset, sort the nodes by increasing total number of cores count, with possibly a condition
- *         that pushes nodes with low disk space to the end of the list? Or a weighted combination of the relative
- *         importance of these two factors? Some randomization? Marking as non available nodes with not enough disk space?
- *         These and other are likely aspects to be played with once the plugin is tested or observed to be running in prod,
- *         don't expect the initial code drop(s) to do all of that.</li>
- *         <li>Iterate over the number of replicas to place (for the current replica type for the current shard):
- *         <ul>
- *             <li>Based on the number of replicas per AZ collected previously, pick the non empty set of nodes having the
- *             lowest number of replicas. Then pick the first node in that set. That's the node the replica is placed one.
- *             Remove the node from the set of available nodes for the given AZ and increase the number of replicas placed
- *             on that AZ.</li>
- *         </ul></li>
- *         <li>During this process, the number of cores on the nodes in general is tracked to take into account placement
- *         decisions so that not all shards decide to put their replicas on the same nodes (they might though if these are
- *         the less loaded nodes).</li>
- *     </ul>
- * </li>
- * </ul>
- *
- * <p>This code is a realistic placement computation, based on a few assumptions. The code is written in such a way to
- * 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>
- *
- * <p>In order to configure this plugin to be used for placement decisions, the following {@code curl} command (or something
- * equivalent) has to be executed once the cluster is already running in order to set
- * the appropriate Zookeeper stored configuration. Replace {@code localhost:8983} by one of your servers' IP address and port.</p>
- *
- * <pre>
- *
-  curl -X POST -H 'Content-type:application/json' -d '{
-    "set-placement-plugin": {
-      "class": "org.apache.solr.cluster.placement.plugins.SamplePluginAffinityReplicaPlacement$Factory",
-      "minimalFreeDiskGB": 10,
-      "deprioritizedFreeDiskGB": 50
-    }
-  }' http://localhost:8983/api/cluster
- * </pre>
- *
- * <p>The consequence will be the creation of an element in the Zookeeper file {@code /clusterprops.json} as follows:</p>
- *
- * <pre>
- *
- * "placement-plugin":{
- *     "class":"org.apache.solr.cluster.placement.plugins.SamplePluginAffinityReplicaPlacement$Factory",
- *     "minimalFreeDiskGB":10,
- *     "deprioritizedFreeDiskGB":50}
- * </pre>
- *
- * <p>In order to delete the placement-plugin section from {@code /clusterprops.json} (and to fallback to either Legacy
- * or rule based placement if configured for a collection), execute:</p>
- *
- * <pre>
- *
-  curl -X POST -H 'Content-type:application/json' -d '{
-    "set-placement-plugin" : null
-  }' http://localhost:8983/api/cluster
- * </pre>
- */
-public class SamplePluginAffinityReplicaPlacement implements PlacementPlugin {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  /**
-   * This factory is instantiated by config from its class name. Using it is the only way to create instances of
-   * {@link SamplePluginAffinityReplicaPlacement}.
-   */
-  static public class Factory implements PlacementPluginFactory {
-
-    /**
-     * 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
-     * having to call a constructor with more parameters (if we were to instantiate the plugin class directly without going
-     * through a factory).
-     */
-    public Factory() {
-    }
-
-    @Override
-    public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
-      final long minimalFreeDiskGB = config.getLongConfig("minimalFreeDiskGB", 20L);
-      final long deprioritizedFreeDiskGB = config.getLongConfig("deprioritizedFreeDiskGB", 100L);
-      return new SamplePluginAffinityReplicaPlacement(minimalFreeDiskGB, deprioritizedFreeDiskGB);
-    }
-  }
-
-
-  /**
-   * <p>Name of the system property on a node indicating which (public cloud) Availability Zone that node is in. The value
-   * is any string, different strings denote different availability zones.
-   *
-   * <p>Nodes on which this system property is not defined are considered being in the same Availability Zone
-   * {@link #UNDEFINED_AVAILABILITY_ZONE} (hopefully the value of this constant is not the name of a real Availability Zone :).
-   */
-  public static final String AVAILABILITY_ZONE_SYSPROP = "availability_zone";
-  /** This is the "AZ" name for nodes that do not define an AZ. Should not match a real AZ name (I think we're safe) */
-  public static final String UNDEFINED_AVAILABILITY_ZONE = "uNd3f1NeD";
-
-  /**
-   * <p>Name of the system property on a node indicating the type of replicas allowed on that node.
-   * The value of that system property is a comma separated list or a single string of value names of
-   * {@link org.apache.solr.cluster.Replica.ReplicaType} (case insensitive). If that property is not defined, that node is
-   * considered accepting all replica types (i.e. undefined is equivalent to {@code "NRT,Pull,tlog"}).
-   *
-   * <p>See {@link #getNodesPerReplicaType}.
-   */
-  public static final String REPLICA_TYPE_SYSPROP = "replica_type";
-
-  /**
-   * 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.
-   */
-  private final 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).
-   */
-  private final long deprioritizedFreeDiskGB;
-
-  /**
-   * The factory has decoded the configuration for the plugin instance and passes it the parameters it needs.
-   */
-  private SamplePluginAffinityReplicaPlacement(long minimalFreeDiskGB, long deprioritizedFreeDiskGB) {
-    this.minimalFreeDiskGB = minimalFreeDiskGB;
-    this.deprioritizedFreeDiskGB = deprioritizedFreeDiskGB;
-  }
-
-  @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
-  public PlacementPlan computePlacement(Cluster cluster, PlacementRequest request, AttributeFetcher attributeFetcher,
-                                        PlacementPlanFactory placementPlanFactory) throws PlacementException {
-    Set<Node> nodes = request.getTargetNodes();
-    SolrCollection solrCollection = request.getCollection();
-
-    // Request all needed attributes
-    attributeFetcher.requestNodeSystemProperty(AVAILABILITY_ZONE_SYSPROP).requestNodeSystemProperty(REPLICA_TYPE_SYSPROP);
-    attributeFetcher.requestNodeCoreCount().requestNodeFreeDisk();
-    attributeFetcher.fetchFrom(nodes);
-    final AttributeValues attrValues = attributeFetcher.fetchAttributes();
-
-    // Split the set of nodes into 3 sets of nodes accepting each replica type (sets can overlap if nodes accept multiple replica types)
-    // These subsets sets are actually maps, because we capture the number of cores (of any replica type) present on each node.
-    // Also get the number of currently existing cores per node, so we can keep update as we place new cores to not end up
-    // always selecting the same node(s).
-    Pair<EnumMap<Replica.ReplicaType, Set<Node>>, Map<Node, Integer>> p = getNodesPerReplicaType(nodes, attrValues);
-
-    EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes = p.first();
-    Map<Node, Integer> coresOnNodes = p.second();
-
-    // All available zones of live nodes. Due to some nodes not being candidates for placement, and some existing replicas
-    // being one availability zones that might be offline (i.e. their nodes are not live), this set might contain zones
-    // on which it is impossible to place replicas. That's ok.
-    ImmutableSet<String> availabilityZones = getZonesFromNodes(nodes, attrValues);
-
-    // Build the replica placement decisions here
-    Set<ReplicaPlacement> replicaPlacements = new HashSet<>();
-
-    // Let's now iterate on all shards to create replicas for and start finding home sweet homes for the replicas
-    for (String shardName : request.getShardNames()) {
-      // Iterate on the replica types in the enum order. We place more strategic replicas first
-      // (NRT is more strategic than TLOG more strategic than PULL). This is in case we eventually decide that less
-      // strategic replica placement impossibility is not a problem that should lead to replica placement computation
-      // failure. Current code does fail if placement is impossible (constraint is at most one replica of a shard on any node).
-      for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
-        makePlacementDecisions(solrCollection, shardName, availabilityZones, replicaType, request.getCountReplicasToCreate(replicaType),
-                attrValues, replicaTypeToNodes, coresOnNodes, placementPlanFactory, replicaPlacements);
-      }
-    }
-
-    return placementPlanFactory.createPlacementPlan(request, replicaPlacements);
-  }
-
-  private ImmutableSet<String> getZonesFromNodes(Set<Node> nodes, final AttributeValues attrValues) {
-    Set<String> azs = new HashSet<>();
-
-    for (Node n : nodes) {
-      azs.add(getNodeAZ(n, attrValues));
-    }
-
-    return ImmutableSet.copyOf(azs);
-  }
-
-  /**
-   * Resolves the AZ of a node and takes care of nodes that have no defined AZ in system property {@link #AVAILABILITY_ZONE_SYSPROP}
-   * to then return {@link #UNDEFINED_AVAILABILITY_ZONE} as the AZ name.
-   */
-  private String getNodeAZ(Node n, final AttributeValues attrValues) {
-    Optional<String> nodeAz = attrValues.getSystemProperty(n, AVAILABILITY_ZONE_SYSPROP);
-    // All nodes with undefined AZ will be considered part of the same AZ. This also works for deployments that do not care about AZ's
-    return nodeAz.orElse(UNDEFINED_AVAILABILITY_ZONE);
-  }
-
-  /**
-   * This class captures an availability zone and the nodes that are legitimate targets for replica placement in that
-   * Availability Zone. Instances are used as values in a {@link TreeMap} in which the total number of already
-   * existing replicas in the AZ is the key. This allows easily picking the set of nodes from which to select a node for
-   * placement in order to balance the number of replicas per AZ. Picking one of the nodes from the set is done using
-   * different criteria unrelated to the Availability Zone (picking the node is based on the {@link CoresAndDiskComparator}
-   * ordering).
-   */
-  private static class AzWithNodes {
-    final String azName;
-    List<Node> availableNodesForPlacement;
-    boolean hasBeenSorted;
-
-    AzWithNodes(String azName, List<Node> availableNodesForPlacement) {
-      this.azName = azName;
-      this.availableNodesForPlacement = availableNodesForPlacement;
-      // Once the list is sorted to an order we're happy with, this flag is set to true to avoid sorting multiple times
-      // unnecessarily.
-      this.hasBeenSorted = false;
-    }
-  }
-
-  /**
-   * Given the set of all nodes on which to do placement and fetched attributes, builds the sets representing
-   * candidate nodes for placement of replicas of each replica type.
-   * These sets are packaged and returned in an EnumMap keyed by replica type (1st member of the Pair).
-   * Also builds the number of existing cores on each node present in the returned EnumMap (2nd member of the returned Pair).
-   * Nodes for which the number of cores is not available for whatever reason are excluded from acceptable candidate nodes
-   * as it would not be possible to make any meaningful placement decisions.
-   * @param nodes all nodes on which this plugin should compute placement
-   * @param attrValues attributes fetched for the nodes. This method uses system property {@link #REPLICA_TYPE_SYSPROP} as
-   *                   well as the number of cores on each node.
-   */
-  private Pair<EnumMap<Replica.ReplicaType, Set<Node>>, Map<Node, Integer>> getNodesPerReplicaType(Set<Node> nodes, final AttributeValues attrValues) {
-    EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes = new EnumMap<>(Replica.ReplicaType.class);
-    Map<Node, Integer> coresOnNodes = Maps.newHashMap();
-
-    for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
-      replicaTypeToNodes.put(replicaType, new HashSet<>());
-    }
-
-    for (Node node : nodes) {
-      // Exclude nodes with unknown or too small disk free space
-      if (attrValues.getFreeDisk(node).isEmpty()) {
-        if (log.isWarnEnabled()) {
-          log.warn("Unknown free disk on node {}, excluding it from placement decisions.", node.getName());
-        }
-        // We rely later on the fact that the free disk optional is present (see CoresAndDiskComparator), be careful it you change anything here.
-        continue;
-      } if (attrValues.getFreeDisk(node).get() < minimalFreeDiskGB) {
-        if (log.isWarnEnabled()) {
-          log.warn("Node {} free disk ({}GB) lower than configured minimum {}GB, excluding it from placement decisions.", node.getName(), attrValues.getFreeDisk(node).get(), minimalFreeDiskGB);
-        }
-        continue;
-      }
-
-      if (attrValues.getCoresCount(node).isEmpty()) {
-        if (log.isWarnEnabled()) {
-          log.warn("Unknown number of cores on node {}, excluding it from placement decisions.", node.getName());
-        }
-        // We rely later on the fact that the number of cores optional is present (see CoresAndDiskComparator), be careful it you change anything here.
-        continue;
-      }
-
-      Integer coresCount = attrValues.getCoresCount(node).get();
-      coresOnNodes.put(node, coresCount);
-
-      String supportedReplicaTypes = attrValues.getSystemProperty(node, REPLICA_TYPE_SYSPROP).isPresent() ? attrValues.getSystemProperty(node, REPLICA_TYPE_SYSPROP).get() : null;
-      // If property not defined or is only whitespace on a node, assuming node can take any replica type
-      if (supportedReplicaTypes == null || supportedReplicaTypes.isBlank()) {
-        for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
-          replicaTypeToNodes.get(rt).add(node);
-        }
-      } else {
-        Set<String> acceptedTypes = Arrays.stream(supportedReplicaTypes.split(",")).map(String::trim).map(s -> s.toLowerCase(Locale.ROOT)).collect(Collectors.toSet());
-        for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
-          if (acceptedTypes.contains(rt.name().toLowerCase(Locale.ROOT))) {
-            replicaTypeToNodes.get(rt).add(node);
-          }
-        }
-      }
-    }
-    return new Pair<>(replicaTypeToNodes, coresOnNodes);
-  }
-
-  /**
-   * <p>Picks nodes from {@code targetNodes} for placing {@code numReplicas} replicas.
-   *
-   * <p>The criteria used in this method are, in this order:
-   * <ol>
-   *     <li>No more than one replica of a given shard on a given node (strictly enforced)</li>
-   *     <li>Balance as much as possible the number of replicas of the given {@link org.apache.solr.cluster.Replica.ReplicaType} over available AZ's.
-   *     This balancing takes into account existing replicas <b>of the corresponding replica type</b>, if any.</li>
-   *     <li>Place replicas is possible on nodes having more than a certain amount of free disk space (note that nodes with a too small
-   *     amount of free disk space were eliminated as placement targets earlier, in {@link #getNodesPerReplicaType}). There's
-   *     a threshold here rather than sorting on the amount of free disk space, because sorting on that value would in
-   *     practice lead to never considering the number of cores on a node.</li>
-   *     <li>Place replicas on nodes having a smaller number of cores (the number of cores considered
-   *     for this decision includes decisions made during the processing of the placement request)</li>
-   * </ol>
-   */
-  @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
-  private void makePlacementDecisions(SolrCollection solrCollection, String shardName, ImmutableSet<String> availabilityZones,
-                                      Replica.ReplicaType replicaType, int numReplicas, final AttributeValues attrValues,
-                                      EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes, Map<Node, Integer> coresOnNodes,
-                                      PlacementPlanFactory placementPlanFactory, Set<ReplicaPlacement> replicaPlacements) throws PlacementException {
-    // Build the set of candidate nodes, i.e. nodes not having (yet) a replica of the given shard
-    Set<Node> candidateNodes = new HashSet<>(replicaTypeToNodes.get(replicaType));
-
-    // Count existing replicas per AZ. We count only instances the type of replica for which we need to do placement. This
-    // can be changed in the loop below if we want to count all replicas for the shard.
-    Map<String, Integer> azToNumReplicas = Maps.newHashMap();
-    // Add all "interesting" AZ's, i.e. AZ's for which there's a chance we can do placement.
-    for (String az : availabilityZones) {
-      azToNumReplicas.put(az, 0);
-    }
-
-    Shard shard = solrCollection.getShard(shardName);
-    if (shard != null) {
-      // shard is non null if we're adding replicas to an already existing collection.
-      // If we're creating the collection, the shards do not exist yet.
-      for (Replica replica : shard.replicas()) {
-        // Nodes already having any type of replica for the shard can't get another replica.
-        candidateNodes.remove(replica.getNode());
-        // The node's AZ has to be counted as having a replica if it has a replica of the same type as the one we need
-        // to place here (remove the "if" below to balance the number of replicas per AZ across all replica types rather
-        // than within each replica type, but then there's a risk that all NRT replicas for example end up on the same AZ).
-        // Note that if in the cluster nodes are configured to accept a single replica type and not multiple ones, the
-        // two options are equivalent (governed by system property AVAILABILITY_ZONE_SYSPROP on each node)
-        if (replica.getType() == replicaType) {
-          final String az = getNodeAZ(replica.getNode(), attrValues);
-          if (azToNumReplicas.containsKey(az)) {
-            // We do not count replicas on AZ's for which we don't have any node to place on because it would not help
-            // the placement decision. If we did want to do that, note the dereferencing below can't be assumed as the
-            // entry will not exist in the map.
-            azToNumReplicas.put(az, azToNumReplicas.get(az) + 1);
-          }
-        }
-      }
-    }
-
-    // We now have the set of real candidate nodes, we've enforced "No more than one replica of a given shard on a given node".
-    // We also counted for the shard and replica type under consideration how many replicas were per AZ, so we can place
-    // (or try to place) replicas on AZ's that have fewer replicas
-
-    // Get the candidate nodes per AZ in order to build (further down) a mapping of AZ to placement candidates.
-    Map<String, List<Node>> nodesPerAz = Maps.newHashMap();
-    for (Node node : candidateNodes) {
-      String nodeAz = getNodeAZ(node, attrValues);
-      List<Node> nodesForAz = nodesPerAz.computeIfAbsent(nodeAz, k -> new ArrayList<>());
-      nodesForAz.add(node);
-    }
-
-    // Build a treeMap sorted by the number of replicas per AZ and including candidates nodes suitable for placement on the
-    // AZ, so we can easily select the next AZ to get a replica assignment and quickly (constant time) decide if placement
-    // on this AZ is possible or not.
-    TreeMultimap<Integer, AzWithNodes> azByExistingReplicas = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());
-    for (Map.Entry<String, List<Node>> e : nodesPerAz.entrySet()) {
-      azByExistingReplicas.put(azToNumReplicas.get(e.getKey()), new AzWithNodes(e.getKey(), e.getValue()));
-    }
-
-    CoresAndDiskComparator coresAndDiskComparator = new CoresAndDiskComparator(attrValues, coresOnNodes, deprioritizedFreeDiskGB);
-
-    // Now we have for each AZ on which we might have a chance of placing a replica, the list of candidate nodes for replicas
-    // (candidate: does not already have a replica of this shard and is in the corresponding AZ).
-    // We must now select those of the nodes on which we actually place the replicas, and will do that based on the total
-    // number of cores already present on these nodes as well as the free disk space.
-    // We sort once by the order related to number of cores and disk space each list of nodes on an AZ. We do not sort all
-    // of them ahead of time because we might be placing a small number of replicas and it might be wasted work.
-    for (int i = 0; i < numReplicas; i++) {
-      // Pick the AZ having the lowest number of replicas for this shard, and if that AZ has available nodes, pick the
-      // most appropriate one (based on number of cores and disk space constraints). In the process, remove entries (AZ's)
-      // that do not have nodes to place replicas on because these are useless to us.
-      Map.Entry<Integer, AzWithNodes> azWithNodesEntry = null;
-      for (Iterator<Map.Entry<Integer, AzWithNodes>> it = azByExistingReplicas.entries().iterator(); it.hasNext(); ) {
-        Map.Entry<Integer, AzWithNodes> entry = it.next();
-        if (!entry.getValue().availableNodesForPlacement.isEmpty()) {
-          azWithNodesEntry = entry;
-          // Remove this entry. Will add it back after a node has been removed from the list of available nodes and the number
-          // of replicas on the AZ has been increased by one (search for "azByExistingReplicas.put" below).
-          it.remove();
-          break;
-        } else {
-          it.remove();
-        }
-      }
-
-      if (azWithNodesEntry == null) {
-        // This can happen because not enough nodes for the placement request or already too many nodes with replicas of
-        // the shard that can't accept new replicas or not enough nodes with enough free disk space.
-        throw new PlacementException("Not enough nodes to place " + numReplicas + " replica(s) of type " + replicaType +
-                " for shard " + shardName + " of collection " + solrCollection.getName());
-      }
-
-      AzWithNodes azWithNodes = azWithNodesEntry.getValue();
-      List<Node> nodes = azWithNodes.availableNodesForPlacement;
-
-      if (!azWithNodes.hasBeenSorted) {
-        // Make sure we do not tend to use always the same nodes (within an AZ) if all conditions are identical (well, this
-        // likely is not the case since after having added a replica to a node its number of cores increases for the next
-        // placement decision, but let's be defensive here, given that multiple concurrent placement decisions might see
-        // the same initial cluster state, and we want placement to be reasonable even in that case without creating an
-        // unnecessary imbalance).
-        // For example, if all nodes have 0 cores and same amount of free disk space, ideally we want to pick a random node
-        // for placement, not always the same one due to some internal ordering.
-        Collections.shuffle(nodes, new Random());
-
-        // Sort by increasing number of cores but pushing nodes with low free disk space to the end of the list
-        nodes.sort(coresAndDiskComparator);
-
-        azWithNodes.hasBeenSorted = true;
-      }
-
-      Node assignTarget = nodes.remove(0);
-
-      // Insert back a corrected entry for the AZ: one more replica living there and one less node that can accept new replicas
-      // (the remaining candidate node list might be empty, in which case it will be cleaned up on the next iteration).
-      azByExistingReplicas.put(azWithNodesEntry.getKey() + 1, azWithNodes);
-
-      // Track that the node has one more core. These values are only used during the current run of the plugin.
-      coresOnNodes.merge(assignTarget, 1, Integer::sum);
-
-      // Register the replica assignment just decided
-      replicaPlacements.add(placementPlanFactory.createReplicaPlacement(solrCollection, shardName, assignTarget, replicaType));
-    }
-  }
-
-  /**
-   * Comparator implementing the placement strategy based on free space and number of cores: we want to place new replicas
-   * on nodes with the less number of cores, but only if they do have enough disk space (expressed as a threshold value).
-   */
-  static class CoresAndDiskComparator implements Comparator<Node> {
-    private final AttributeValues attrValues;
-    private final Map<Node, Integer> coresOnNodes;
-    private final long deprioritizedFreeDiskGB;
-
-
-    /**
-     * The data we sort on is not part of the {@link Node} instances but has to be retrieved from the attributes and configuration.
-     * The number of cores per node is passed in a map whereas the free disk is fetched from the attributes due to the
-     * fact that we update the number of cores per node as we do allocations, but we do not update the free disk. The
-     * attrValues correpsonding to the number of cores per node are the initial values, but we want to comapre the actual
-     * value taking into account placement decisions already made during the current execution of the placement plugin.
-     */
-    CoresAndDiskComparator(AttributeValues attrValues, Map<Node, Integer> coresOnNodes, long deprioritizedFreeDiskGB) {
-      this.attrValues = attrValues;
-      this.coresOnNodes = coresOnNodes;
-      this.deprioritizedFreeDiskGB = deprioritizedFreeDiskGB;
-    }
-
-    @Override
-    public int compare(Node a, Node b) {
-      // Note all nodes do have free disk defined. This has been verified earlier.
-      boolean aHasLowFreeSpace = attrValues.getFreeDisk(a).get() < deprioritizedFreeDiskGB;
-      boolean bHasLowFreeSpace = attrValues.getFreeDisk(b).get() < deprioritizedFreeDiskGB;
-      if (aHasLowFreeSpace != bHasLowFreeSpace) {
-        // A node with low free space should be considered > node with high free space since it needs to come later in sort order
-        return Boolean.compare(aHasLowFreeSpace, bHasLowFreeSpace);
-      }
-      // The ordering on the number of cores is the natural order.
-      return Integer.compare(coresOnNodes.get(a), coresOnNodes.get(b));
-    }
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginMinimizeCores.java b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginMinimizeCores.java
deleted file mode 100644
index 54520fc..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginMinimizeCores.java
+++ /dev/null
@@ -1,138 +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.plugins;
-
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.Map;
-
-import com.google.common.collect.Ordering;
-import com.google.common.collect.TreeMultimap;
-import org.apache.solr.cluster.Cluster;
-import org.apache.solr.cluster.Node;
-import org.apache.solr.cluster.Replica;
-import org.apache.solr.cluster.SolrCollection;
-import org.apache.solr.cluster.placement.*;
-import org.apache.solr.common.util.SuppressForbidden;
-
-/**
- * <p>Implements placing replicas to minimize number of cores per {@link Node}, while not placing two replicas of the same
- * shard on the same node.</p>
- *
- * <p>Warning: not really tested. See {@link SamplePluginAffinityReplicaPlacement} for a more realistic example.</p>
- */
-public class SamplePluginMinimizeCores implements PlacementPlugin {
-
-  private final PlacementPluginConfig config;
-
-  private SamplePluginMinimizeCores(PlacementPluginConfig config) {
-    this.config = config;
-  }
-
-  static public class Factory implements PlacementPluginFactory {
-
-    /**
-     * Empty public constructor is used to instantiate this factory based on configuration in solr.xml, element
-     * {@code <placementPluginFactory>} in element {@code <solrcloud>}.
-     */
-    public Factory() {
-    }
-
-    @Override
-    public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
-      return new SamplePluginMinimizeCores(config);
-    }
-  }
-
-  @SuppressForbidden(reason = "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
-  public PlacementPlan computePlacement(Cluster cluster, PlacementRequest request, AttributeFetcher attributeFetcher,
-                                        PlacementPlanFactory placementPlanFactory) throws PlacementException {
-    int totalReplicasPerShard = 0;
-    for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
-      totalReplicasPerShard += request.getCountReplicasToCreate(rt);
-    }
-
-    if (cluster.getLiveNodes().size() < totalReplicasPerShard) {
-      throw new PlacementException("Cluster size too small for number of replicas per shard");
-    }
-
-    // Get number of cores on each Node
-    TreeMultimap<Integer, Node> nodesByCores = TreeMultimap.create(Comparator.naturalOrder(), Ordering.arbitrary());
-
-    Set<Node> nodes = request.getTargetNodes();
-
-    attributeFetcher.requestNodeCoreCount();
-    attributeFetcher.fetchFrom(nodes);
-    AttributeValues attrValues = attributeFetcher.fetchAttributes();
-
-
-    // Get the number of cores on each node and sort the nodes by increasing number of cores
-    for (Node node : nodes) {
-      if (attrValues.getCoresCount(node).isEmpty()) {
-        throw new PlacementException("Can't get number of cores in " + node);
-      }
-      nodesByCores.put(attrValues.getCoresCount(node).get(), node);
-    }
-
-    Set<ReplicaPlacement> replicaPlacements = new HashSet<>(totalReplicasPerShard * request.getShardNames().size());
-
-    // Now place all replicas of all shards on nodes, by placing on nodes with the smallest number of cores and taking
-    // into account replicas placed during this computation. Note that for each shard we must place replicas on different
-    // nodes, when moving to the next shard we use the nodes sorted by their updated number of cores (due to replica
-    // placements for previous shards).
-    for (String shardName : request.getShardNames()) {
-      // Assign replicas based on the sort order of the nodesByCores tree multimap to put replicas on nodes with less
-      // cores first. We only need totalReplicasPerShard nodes given that's the number of replicas to place.
-      // We assign based on the passed nodeEntriesToAssign list so the right nodes get replicas.
-      ArrayList<Map.Entry<Integer, Node>> nodeEntriesToAssign = new ArrayList<>(totalReplicasPerShard);
-      Iterator<Map.Entry<Integer, Node>> treeIterator = nodesByCores.entries().iterator();
-      for (int i = 0; i < totalReplicasPerShard; i++) {
-        nodeEntriesToAssign.add(treeIterator.next());
-      }
-
-      // Update the number of cores each node will have once the assignments below got executed so the next shard picks the
-      // lowest loaded nodes for its replicas.
-      for (Map.Entry<Integer, Node> e : nodeEntriesToAssign) {
-        int coreCount = e.getKey();
-        Node node = e.getValue();
-        nodesByCores.remove(coreCount, node);
-        nodesByCores.put(coreCount + 1, node);
-      }
-
-      for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
-        placeReplicas(request.getCollection(), nodeEntriesToAssign, placementPlanFactory, replicaPlacements, shardName, request, replicaType);
-      }
-    }
-
-    return placementPlanFactory.createPlacementPlan(request, replicaPlacements);
-  }
-
-  private void placeReplicas(SolrCollection solrCollection, ArrayList<Map.Entry<Integer, Node>> nodeEntriesToAssign,
-                             PlacementPlanFactory placementPlanFactory, Set<ReplicaPlacement> replicaPlacements,
-                             String shardName, PlacementRequest request, Replica.ReplicaType replicaType) {
-    for (int replica = 0; replica < request.getCountReplicasToCreate(replicaType); replica++) {
-      final Map.Entry<Integer, Node> entry = nodeEntriesToAssign.remove(0);
-      final Node node = entry.getValue();
-
-      replicaPlacements.add(placementPlanFactory.createReplicaPlacement(solrCollection, shardName, node, replicaType));
-    }
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginRandomPlacement.java b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginRandomPlacement.java
deleted file mode 100644
index eecb57f..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/placement/plugins/SamplePluginRandomPlacement.java
+++ /dev/null
@@ -1,88 +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.plugins;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Random;
-import java.util.Set;
-
-import org.apache.solr.cluster.Cluster;
-import org.apache.solr.cluster.Node;
-import org.apache.solr.cluster.Replica;
-import org.apache.solr.cluster.SolrCollection;
-import org.apache.solr.cluster.placement.*;
-
-/**
- * Implements random placement for new collection creation while preventing two replicas of same shard from being placed on same node.
- *
- * <p>Warning: not really tested. See {@link SamplePluginAffinityReplicaPlacement} for a more realistic example.</p>
- */
-public class SamplePluginRandomPlacement implements PlacementPlugin {
-
-  private final PlacementPluginConfig config;
-
-  private SamplePluginRandomPlacement(PlacementPluginConfig config) {
-    this.config = config;
-  }
-
-  static public class Factory implements PlacementPluginFactory {
-    @Override
-    public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
-      return new SamplePluginRandomPlacement(config);
-    }
-  }
-
-  public PlacementPlan computePlacement(Cluster cluster, PlacementRequest request, AttributeFetcher attributeFetcher,
-                                        PlacementPlanFactory placementPlanFactory) throws PlacementException {
-    int totalReplicasPerShard = 0;
-    for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
-      totalReplicasPerShard += request.getCountReplicasToCreate(rt);
-    }
-
-    if (cluster.getLiveNodes().size() < totalReplicasPerShard) {
-      throw new PlacementException("Cluster size too small for number of replicas per shard");
-    }
-
-    Set<ReplicaPlacement> replicaPlacements = new HashSet<>(totalReplicasPerShard * request.getShardNames().size());
-
-    // Now place randomly all replicas of all shards on available nodes
-    for (String shardName : request.getShardNames()) {
-      // Shuffle the nodes for each shard so that replicas for a shard are placed on distinct yet random nodes
-      ArrayList<Node> nodesToAssign = new ArrayList<>(cluster.getLiveNodes());
-      Collections.shuffle(nodesToAssign, new Random());
-
-      for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
-        placeForReplicaType(request.getCollection(), nodesToAssign, placementPlanFactory, replicaPlacements, shardName, request, rt);
-      }
-    }
-
-    return placementPlanFactory.createPlacementPlan(request, replicaPlacements);
-  }
-
-  private void placeForReplicaType(SolrCollection solrCollection, ArrayList<Node> nodesToAssign, PlacementPlanFactory placementPlanFactory,
-                                   Set<ReplicaPlacement> replicaPlacements,
-                                   String shardName, PlacementRequest request, Replica.ReplicaType replicaType) {
-    for (int replica = 0; replica < request.getCountReplicasToCreate(replicaType); replica++) {
-      Node node = nodesToAssign.remove(0);
-
-      replicaPlacements.add(placementPlanFactory.createReplicaPlacement(solrCollection, shardName, node, replicaType));
-    }
-  }
-}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
similarity index 93%
rename from solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java
rename to solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
index e6d7cd4..35bfcfc 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
@@ -22,7 +22,7 @@ import org.apache.solr.cluster.Node;
 import org.apache.solr.cluster.Shard;
 import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.*;
-import org.apache.solr.cluster.placement.plugins.SamplePluginAffinityReplicaPlacement;
+import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -31,9 +31,9 @@ import java.util.Map;
 import java.util.Set;
 
 /**
- * Unit test for {@link SamplePluginAffinityReplicaPlacement}
+ * Unit test for {@link AffinityPlacementFactory}
  */
-public class SamplePluginAffinityReplicaPlacementTest extends Assert {
+public class AffinityPlacementFactoryTest extends Assert {
 
     private static PlacementPlugin plugin;
 
@@ -41,7 +41,7 @@ public class SamplePluginAffinityReplicaPlacementTest extends Assert {
     public static void setupPlugin() {
         PlacementPluginConfig config = PlacementPluginConfigImpl.createConfigFromProperties(
                 Map.of("minimalFreeDiskGB", 10L, "deprioritizedFreeDiskGB", 50L));
-        plugin = new SamplePluginAffinityReplicaPlacement.Factory().createPluginInstance(config);
+        plugin = new AffinityPlacementFactory().createPluginInstance(config);
     }
 
     @Test
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 54cd081..2acad7e 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
@@ -6,7 +6,7 @@ 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.plugins.SamplePluginMinimizeCores;
+import org.apache.solr.cluster.placement.plugins.MinimizeCoresPlacementFactory;
 import org.apache.solr.common.cloud.ClusterProperties;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
@@ -55,7 +55,7 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
 
   @Test
   public void testMinimizeCores() throws Exception {
-    Map<String, Object> config = Map.of(PlacementPluginConfig.FACTORY_CLASS, SamplePluginMinimizeCores.Factory.class.getName());
+    Map<String, Object> config = Map.of(PlacementPluginConfig.FACTORY_CLASS, MinimizeCoresPlacementFactory.class.getName());
     V2Request req = new V2Request.Builder("/cluster")
         .forceV2(true)
         .POST()

[lucene] 02/33: solr-15004: initial example (WIP and buggy) of plugin test

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 2267f3d1640f636fe4933a430bf291403eb2d892
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Wed Nov 18 01:46:04 2020 +0100

    solr-15004: initial example (WIP and buggy) of plugin test
---
 .../impl/PlacementPluginAssignStrategy.java        |   4 +-
 .../placement/impl/PlacementRequestImpl.java       |  14 +-
 .../placement/impl/AttributeFetcherForTest.java    |  93 +++++++
 .../placement/impl/ClusterAbstractionsForTest.java | 292 +++++++++++++++++++++
 .../cluster/placement/impl/PluginTestHelper.java   |  71 +++++
 .../SamplePluginAffinityReplicaPlacementTest.java  |  92 +++++++
 .../impl/SimpleClusterAbstractionsTest.java        |   1 +
 7 files changed, 558 insertions(+), 9 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginAssignStrategy.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginAssignStrategy.java
index 0bbf4e0..c4c5667 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginAssignStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginAssignStrategy.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.cloud.api.collections.Assign;
 import org.apache.solr.cluster.Cluster;
+import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.PlacementException;
 import org.apache.solr.cluster.placement.PlacementPlugin;
 import org.apache.solr.cluster.placement.PlacementPlan;
@@ -53,8 +54,9 @@ public class PlacementPluginAssignStrategy implements Assign.AssignStrategy {
       throws Assign.AssignmentException, IOException, InterruptedException {
 
     Cluster cluster = new SimpleClusterAbstractionsImpl.ClusterImpl(solrCloudManager);
+    SolrCollection solrCollection = new SimpleClusterAbstractionsImpl.SolrCollectionImpl(collection);
 
-    PlacementRequestImpl placementRequest = PlacementRequestImpl.toPlacementRequest(cluster, collection, assignRequest);
+    PlacementRequestImpl placementRequest = PlacementRequestImpl.toPlacementRequest(cluster, solrCollection, assignRequest);
 
     final PlacementPlan placementPlan;
     try {
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
index 80cf6c5..82a26d1 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
@@ -26,8 +26,7 @@ import org.apache.solr.cluster.Cluster;
 import org.apache.solr.cluster.Node;
 import org.apache.solr.cluster.Replica;
 import org.apache.solr.cluster.SolrCollection;
-import org.apache.solr.cluster.placement.*;
-import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.cluster.placement.PlacementRequest;
 
 public class PlacementRequestImpl implements PlacementRequest {
   private final SolrCollection solrCollection;
@@ -35,7 +34,7 @@ public class PlacementRequestImpl implements PlacementRequest {
   private final Set<Node> targetNodes;
   private final EnumMap<Replica.ReplicaType, Integer> countReplicas = new EnumMap<>(Replica.ReplicaType.class);
 
-  private PlacementRequestImpl(SolrCollection solrCollection,
+  PlacementRequestImpl(SolrCollection solrCollection,
                                Set<String> shardNames, Set<Node> targetNodes,
                                int countNrtReplicas, int countTlogReplicas, int countPullReplicas) {
     this.solrCollection = solrCollection;
@@ -72,12 +71,11 @@ public class PlacementRequestImpl implements PlacementRequest {
    * Returns a {@link PlacementRequest} that can be consumed by a plugin based on an internal Assign.AssignRequest
    * for adding replicas + additional info (upon creation of a new collection or adding replicas to an existing one).
    */
-  static PlacementRequestImpl toPlacementRequest(Cluster cluster, DocCollection docCollection,
+  static PlacementRequestImpl toPlacementRequest(Cluster cluster, SolrCollection solrCollection,
                                                  Assign.AssignRequest assignRequest) throws Assign.AssignmentException {
-    SolrCollection solrCollection = new SimpleClusterAbstractionsImpl.SolrCollectionImpl(docCollection);
     Set<String> shardNames = new HashSet<>(assignRequest.shardNames);
     if (shardNames.size() < 1) {
-      throw new Assign.AssignmentException("Bad assign request: no shards specified for collection " + docCollection.getName());
+      throw new Assign.AssignmentException("Bad assign request: no shards specified for collection " + solrCollection.getName());
     }
 
     final Set<Node> nodes;
@@ -85,12 +83,12 @@ public class PlacementRequestImpl implements PlacementRequest {
     if (assignRequest.nodes != null) {
       nodes = SimpleClusterAbstractionsImpl.NodeImpl.getNodes(assignRequest.nodes);
       if (nodes.isEmpty()) {
-        throw new Assign.AssignmentException("Bad assign request: empty list of nodes for collection " + docCollection.getName());
+        throw new Assign.AssignmentException("Bad assign request: empty list of nodes for collection " + solrCollection.getName());
       }
     } else {
       nodes = cluster.getLiveNodes();
       if (nodes.isEmpty()) {
-        throw new Assign.AssignmentException("Impossible assign request: no live nodes for collection " + docCollection.getName());
+        throw new Assign.AssignmentException("Impossible assign request: no live nodes for collection " + solrCollection.getName());
       }
     }
 
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java
new file mode 100644
index 0000000..58005f7
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.Node;
+import org.apache.solr.cluster.placement.AttributeFetcher;
+import org.apache.solr.cluster.placement.AttributeValues;
+
+import java.util.Set;
+
+public class AttributeFetcherForTest implements AttributeFetcher {
+
+    private final AttributeValues attributeValues;
+
+    AttributeFetcherForTest(AttributeValues attributeValues) {
+        this.attributeValues = attributeValues;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeCoreCount() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeDiskType() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeFreeDisk() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeTotalDisk() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeHeapUsage() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeSystemLoadAverage() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeSystemProperty(String name) {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeEnvironmentVariable(String name) {
+        throw new UnsupportedOperationException("Not yet implemented...");
+    }
+
+    @Override
+    public AttributeFetcher requestNodeMetric(String metricName, NodeMetricRegistry registry) {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher fetchFrom(Set<Node> nodes) {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestMetric(String scope, String metricName) {
+        throw new UnsupportedOperationException("Not yet implemented...");
+    }
+
+    @Override
+    public AttributeValues fetchAttributes() {
+        return attributeValues;
+    }
+}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
new file mode 100644
index 0000000..188e3c3
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
@@ -0,0 +1,292 @@
+/*
+ * 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.*;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.*;
+import java.util.stream.Collectors;
+
+/**
+ * Cluster abstractions independent of any internal SolrCloud abstractions to use in tests (of plugin code).
+ */
+class ClusterAbstractionsForTest {
+
+    static class ClusterImpl implements Cluster {
+        private final Set<Node> liveNodes;
+        private final Map<String, SolrCollection> collections;
+
+        ClusterImpl(Set<Node> liveNodes, Map<String, SolrCollection> collections) throws IOException {
+            this.liveNodes = liveNodes;
+            this.collections = collections;
+        }
+
+        @Override
+        public Set<Node> getLiveNodes() {
+            return liveNodes;
+        }
+
+        @Override
+        public SolrCollection getCollection(String collectionName) {
+            return collections.get(collectionName);
+        }
+
+        @Override
+        @Nonnull
+        public Iterator<SolrCollection> iterator() {
+            return collections.values().iterator();
+        }
+
+        @Override
+        public Iterable<SolrCollection> collections() {
+            return ClusterImpl.this::iterator;
+        }
+    }
+
+
+    static class NodeImpl implements Node {
+        public final String nodeName;
+
+        /**
+         * Transforms a collection of node names into a set of {@link Node} instances.
+         */
+        static Set<Node> getNodes(Collection<String> nodeNames) {
+            return nodeNames.stream().map(NodeImpl::new).collect(Collectors.toSet());
+        }
+
+        NodeImpl(String nodeName) {
+            this.nodeName = nodeName;
+        }
+
+        @Override
+        public String getName() {
+            return nodeName;
+        }
+
+        @Override
+        public String toString() {
+            return getClass().getSimpleName() + "(" + getName() + ")";
+        }
+
+        /**
+         * This class ends up as a key in Maps in {@link org.apache.solr.cluster.placement.AttributeValues}.
+         * It is important to implement this method comparing node names given that new instances of {@link Node} are created
+         * with names equal to existing instances (See {@link ReplicaImpl} constructor).
+         */
+        public boolean equals(Object obj) {
+            if (obj == null) { return false; }
+            if (obj == this) { return true; }
+            if (obj.getClass() != getClass()) { return false; }
+            NodeImpl other = (NodeImpl) obj;
+            return Objects.equals(this.nodeName, other.nodeName);
+        }
+
+        public int hashCode() {
+            return Objects.hashCode(nodeName);
+        }
+    }
+
+
+    static class SolrCollectionImpl implements SolrCollection {
+        private final String collectionName;
+        /** Map from {@link Shard#getShardName()} to {@link Shard} */
+        private Map<String, Shard> shards;
+        private final Map<String, String> customProperties;
+
+        SolrCollectionImpl(String collectionName, Map<String, String> customProperties) {
+            this.collectionName = collectionName;
+            this.customProperties = customProperties;
+        }
+
+        /**
+         * Setting the shards has to happen (in tests) after creating the collection because shards reference the collection
+         */
+        void setShards(Map<String, Shard> shards) {
+            this.shards = shards;
+        }
+
+        @Override
+        public String getName() {
+            return collectionName;
+        }
+
+        @Override
+        public Shard getShard(String name) {
+            return shards.get(name);
+        }
+
+        @Override
+        @Nonnull
+        public Iterator<Shard> iterator() {
+            return shards.values().iterator();
+        }
+
+        @Override
+        public Iterable<Shard> shards() {
+            return SolrCollectionImpl.this::iterator;
+        }
+
+        @Override
+        public String getCustomProperty(String customPropertyName) {
+            return customProperties.get(customPropertyName);
+        }
+    }
+
+
+    static class ShardImpl implements Shard {
+        private final String shardName;
+        private final SolrCollection collection;
+        private final ShardState shardState;
+        private Map<String, Replica> replicas;
+        private Replica leader;
+
+        ShardImpl(String shardName, SolrCollection collection, ShardState shardState) {
+            this.shardName = shardName;
+            this.collection = collection;
+            this.shardState = shardState;
+        }
+
+        /**
+         * Setting the replicas has to happen (in tests) after creating the shard because replicas reference the shard
+         */
+        void setReplicas(Map<String, Replica> replicas, Replica leader) {
+            this.replicas = replicas;
+            this.leader = leader;
+        }
+
+        @Override
+        public String getShardName() {
+            return shardName;
+        }
+
+        @Override
+        public SolrCollection getCollection() {
+            return collection;
+        }
+
+        @Override
+        public Replica getReplica(String name) {
+            return replicas.get(name);
+        }
+
+        @Override
+        @Nonnull
+        public Iterator<Replica> iterator() {
+            return replicas.values().iterator();
+        }
+
+        @Override
+        public Iterable<Replica> replicas() {
+            return ShardImpl.this::iterator;
+        }
+
+        @Override
+        public Replica getLeader() {
+            return leader;
+        }
+
+        @Override
+        public ShardState getState() {
+            return shardState;
+        }
+
+        public boolean equals(Object obj) {
+            if (obj == null) { return false; }
+            if (obj == this) { return true; }
+            if (obj.getClass() != getClass()) { return false; }
+            ShardImpl other = (ShardImpl) obj;
+            return Objects.equals(this.shardName, other.shardName)
+                    && Objects.equals(this.collection, other.collection)
+                    && Objects.equals(this.shardState, other.shardState)
+                    && Objects.equals(this.replicas, other.replicas)
+                    && Objects.equals(this.leader, other.leader);
+        }
+
+        public int hashCode() {
+            return Objects.hash(shardName, collection, shardState);
+        }
+    }
+
+
+    static class ReplicaImpl implements Replica {
+        private final String replicaName;
+        private final String coreName;
+        private final Shard shard;
+        private final ReplicaType replicaType;
+        private final ReplicaState replicaState;
+        private final Node node;
+
+        ReplicaImpl(String replicaName, String coreName, Shard shard, ReplicaType replicaType, ReplicaState replicaState, Node node) {
+            this.replicaName = replicaName;
+            this.coreName = coreName;
+            this.shard = shard;
+            this.replicaType = replicaType;
+            this.replicaState = replicaState;
+            this.node = node;
+        }
+
+        @Override
+        public Shard getShard() {
+            return shard;
+        }
+
+        @Override
+        public ReplicaType getType() {
+            return replicaType;
+        }
+
+        @Override
+        public ReplicaState getState() {
+            return replicaState;
+        }
+
+        @Override
+        public String getReplicaName() {
+            return replicaName;
+        }
+
+        @Override
+        public String getCoreName() {
+            return coreName;
+        }
+
+        @Override
+        public Node getNode() {
+            return node;
+        }
+
+        public boolean equals(Object obj) {
+            if (obj == null) { return false; }
+            if (obj == this) { return true; }
+            if (obj.getClass() != getClass()) { return false; }
+            ReplicaImpl other = (ReplicaImpl) obj;
+            return Objects.equals(this.replicaName, other.replicaName)
+                    && Objects.equals(this.coreName, other.coreName)
+                    && Objects.equals(this.shard, other.shard)
+                    && Objects.equals(this.replicaType, other.replicaType)
+                    && Objects.equals(this.replicaState, other.replicaState)
+                    && Objects.equals(this.node, other.node);
+        }
+
+        public int hashCode() {
+            return Objects.hash(replicaName, coreName, shard, replicaType, replicaState, node);
+        }
+    }
+}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
new file mode 100644
index 0000000..13ef70d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
@@ -0,0 +1,71 @@
+/*
+ * 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.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.Shard;
+import org.apache.solr.cluster.SolrCollection;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+public class PluginTestHelper {
+
+    /**
+     * Builds the representation of shards for a collection, based on the number of shards and replicas for each to create.
+     * The replicas are allocated to the provided nodes in a round robin way. The leader is set to the last replica of each shard.
+     */
+    static Map<String, Shard> createShardsAndReplicas(SolrCollection collection, int numShards, int numNrtReplicas, Set<Node> nodes) {
+        Iterator<Node> nodeIterator = nodes.iterator();
+
+        Map<String, Shard> shards = new HashMap<>();
+
+        for (int s = 0; s < numShards; s++) {
+            String shardName = collection.getName() + "_s" + s;
+
+            ClusterAbstractionsForTest.ShardImpl shard = new ClusterAbstractionsForTest.ShardImpl(shardName, collection, Shard.ShardState.ACTIVE);
+
+            Map<String, Replica> replicas = new HashMap<>();
+            Replica leader = null;
+            for (int r = 0; r < numNrtReplicas; r++) {
+                String replicaName = shardName + "_r" + r;
+                String coreName = replicaName + "_c";
+                final Node node;
+                if (!nodeIterator.hasNext()) {
+                    nodeIterator = nodes.iterator();
+                }
+                // If the nodes set is empty, this call will fail
+                node = nodeIterator.next();
+
+                Replica replica = new ClusterAbstractionsForTest.ReplicaImpl(replicaName, coreName, shard, Replica.ReplicaType.NRT, Replica.ReplicaState.ACTIVE, node);
+
+                replicas.put(replica.getReplicaName(), replica);
+                leader = replica;
+            }
+
+            shard.setReplicas(replicas, leader);
+
+            shards.put(shard.getShardName(), shard);
+        }
+
+        return shards;
+    }
+}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java
new file mode 100644
index 0000000..7c7dee5
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.Cluster;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Shard;
+import org.apache.solr.cluster.placement.*;
+import org.apache.solr.cluster.placement.plugins.SamplePluginAffinityReplicaPlacement;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Unit test for {@link SamplePluginAffinityReplicaPlacement}
+ */
+public class SamplePluginAffinityReplicaPlacementTest extends Assert {
+
+    private static PlacementPlugin plugin;
+
+    @BeforeClass
+    public static void setupCluster() {
+        PlacementPluginConfig config = PlacementPluginConfigImpl.createConfigFromProperties(
+                Map.of("minimalFreeDiskGB", 10L, "deprioritizedFreeDiskGB", 50L));
+        plugin = new SamplePluginAffinityReplicaPlacement.Factory().createPluginInstance(config);
+    }
+
+    // FIXME IG: this test fails (expecting placement on node1 but placement happens on node2)
+    @Test
+    public void testBasicPlacementNewCollection() throws Exception {
+        testBasicPlacementInternal(false);
+    }
+
+    @Test
+    public void testBasicPlacementExistingCollection() throws Exception {
+        testBasicPlacementInternal(true);
+    }
+
+    /**
+     * When this test places a replica for a new collection, it should pick the node with less cores.<p>
+     *
+     * When it places a replica for an existing collection, it should pick the node with more cores that doesn't already have a replica for the shard.
+     */
+    private void testBasicPlacementInternal(boolean hasExistingCollection) throws Exception {
+        String collectionName = "testCollection";
+
+        Node node1 = new ClusterAbstractionsForTest.NodeImpl("node1");
+        Node node2 = new ClusterAbstractionsForTest.NodeImpl("node2");
+//        Set<Node> liveNodes = ClusterAbstractionsForTest.NodeImpl.getNodes(Set.of("node1", "node2", "node3"));
+        Set<Node> liveNodes = Set.of(node1, node2);
+        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(collectionName, Map.of());
+        // An existing collection with a single replica on node 1
+        Map<String, Shard> shards = PluginTestHelper.createShardsAndReplicas(solrCollection, 1, 1, Set.of(node1));
+        solrCollection.setShards(shards);
+
+        Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, hasExistingCollection ? Map.of(solrCollection.getName(), solrCollection) : Map.of());
+        // Place a new replica for the (only) existing shard of the collection
+        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, Set.of(shards.keySet().iterator().next()), liveNodes, 1, 0, 0);
+        // More cores on node2
+        Map<Node, Integer> nodeToCoreCount = Map.of(node1, 1, node2, 10);
+        // A lot of free disk on the two nodes
+        final Map<Node, Long> nodeToFreeDisk = Map.of(node1, 100L, node2, 100L);
+        AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
+        AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
+        PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+
+        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+
+
+        assertEquals(1, pp.getReplicaPlacements().size());
+        ReplicaPlacement rp = pp.getReplicaPlacements().iterator().next();
+        assertEquals(hasExistingCollection ? node2 : node1, rp.getNode());
+    }
+}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsTest.java
index eab2785..5e58779 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsTest.java
@@ -14,6 +14,7 @@
  * 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.client.solrj.cloud.SolrCloudManager;

[lucene] 03/33: SOLR-15004: fix failure in SamplePluginAffinityReplicaPlacementTest

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 023ebc3b4786de042bdc0fe52db34ef9ca7a1ce9
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Thu Nov 19 01:18:32 2020 +0100

    SOLR-15004: fix failure in SamplePluginAffinityReplicaPlacementTest
---
 .../SamplePluginAffinityReplicaPlacementTest.java  | 25 ++++++++++++++++------
 1 file changed, 18 insertions(+), 7 deletions(-)

diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java
index 7c7dee5..e6d7cd4 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java
@@ -20,6 +20,7 @@ package org.apache.solr.cluster.placement.impl;
 import org.apache.solr.cluster.Cluster;
 import org.apache.solr.cluster.Node;
 import org.apache.solr.cluster.Shard;
+import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.*;
 import org.apache.solr.cluster.placement.plugins.SamplePluginAffinityReplicaPlacement;
 import org.junit.Assert;
@@ -37,13 +38,12 @@ public class SamplePluginAffinityReplicaPlacementTest extends Assert {
     private static PlacementPlugin plugin;
 
     @BeforeClass
-    public static void setupCluster() {
+    public static void setupPlugin() {
         PlacementPluginConfig config = PlacementPluginConfigImpl.createConfigFromProperties(
                 Map.of("minimalFreeDiskGB", 10L, "deprioritizedFreeDiskGB", 50L));
         plugin = new SamplePluginAffinityReplicaPlacement.Factory().createPluginInstance(config);
     }
 
-    // FIXME IG: this test fails (expecting placement on node1 but placement happens on node2)
     @Test
     public void testBasicPlacementNewCollection() throws Exception {
         testBasicPlacementInternal(false);
@@ -64,14 +64,25 @@ public class SamplePluginAffinityReplicaPlacementTest extends Assert {
 
         Node node1 = new ClusterAbstractionsForTest.NodeImpl("node1");
         Node node2 = new ClusterAbstractionsForTest.NodeImpl("node2");
-//        Set<Node> liveNodes = ClusterAbstractionsForTest.NodeImpl.getNodes(Set.of("node1", "node2", "node3"));
         Set<Node> liveNodes = Set.of(node1, node2);
+
         ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(collectionName, Map.of());
-        // An existing collection with a single replica on node 1
-        Map<String, Shard> shards = PluginTestHelper.createShardsAndReplicas(solrCollection, 1, 1, Set.of(node1));
-        solrCollection.setShards(shards);
+        // Make sure new collections are not visible in the cluster state and existing ones are
+        final Map<String, SolrCollection> clusterCollections;
+        final Map<String, Shard> shards;
+        if (hasExistingCollection) {
+            // An existing collection with a single replica on node 1. Note that new collections already exist by the time the plugin is called, but are empty
+            shards = PluginTestHelper.createShardsAndReplicas(solrCollection, 1, 1, Set.of(node1));
+            solrCollection.setShards(shards);
+            clusterCollections = Map.of(solrCollection.getName(), solrCollection);
+        } else {
+            // A new collection has the shards defined ok but no replicas
+            shards = PluginTestHelper.createShardsAndReplicas(solrCollection, 1, 0, Set.of());
+            solrCollection.setShards(shards);
+            clusterCollections = Map.of();
+        }
 
-        Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, hasExistingCollection ? Map.of(solrCollection.getName(), solrCollection) : Map.of());
+        Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, clusterCollections);
         // Place a new replica for the (only) existing shard of the collection
         PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, Set.of(shards.keySet().iterator().next()), liveNodes, 1, 0, 0);
         // More cores on node2

[lucene] 26/33: SOLR-15016: Fix precommit issues.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 947db7a06fcdf315af60ed1701283b01d416c852
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Wed Dec 2 15:18:41 2020 +0100

    SOLR-15016: Fix precommit issues.
---
 .../events/impl/ClusterEventProducerFactory.java    |  4 ++--
 .../impl/PlacementPluginFactoryLoader.java          | 21 +++++++++++++++++++--
 .../placement/plugins/AffinityPlacementConfig.java  | 19 +++++++++++++++++--
 .../java/org/apache/solr/core/CoreContainer.java    |  1 -
 .../plugins/AffinityPlacementFactoryTest.java       |  1 -
 5 files changed, 38 insertions(+), 8 deletions(-)

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 609f65c..e79341d 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
@@ -138,7 +138,7 @@ public class ClusterEventProducerFactory extends ClusterEventProducerBase {
                   " using existing implementation {}", cc.getClusterEventProducer().getClass().getName());
             }
           } else {
-            log.warn("Ignoring ClusterEventProducer config with non-standard name: " + plugin.getInfo());
+            log.warn("Ignoring ClusterEventProducer config with non-standard name: {}", plugin.getInfo());
           }
         }
       }
@@ -163,7 +163,7 @@ public class ClusterEventProducerFactory extends ClusterEventProducerBase {
                   " using existing implementation {}", cc.getClusterEventProducer().getClass().getName());
             }
           } else {
-            log.warn("Ignoring ClusterEventProducer config with non-standard name: " + plugin.getInfo());
+            log.warn("Ignoring ClusterEventProducer config with non-standard name: {}", plugin.getInfo());
           }
         }
       }
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 46e6435..523fa17 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
@@ -1,3 +1,20 @@
+/*
+ * 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.api.ContainerPluginsRegistry;
@@ -32,7 +49,7 @@ public class PlacementPluginFactoryLoader {
           if (PlacementPluginFactory.PLUGIN_NAME.equals(plugin.getInfo().name)) {
             pluginFactory.setDelegate((PlacementPluginFactory) instance);
           } else {
-            log.warn("Ignoring PlacementPluginFactory plugin with non-standard name: " + plugin.getInfo());
+            log.warn("Ignoring PlacementPluginFactory plugin with non-standard name: {}", plugin.getInfo());
           }
         }
       }
@@ -47,7 +64,7 @@ public class PlacementPluginFactoryLoader {
           if (PlacementPluginFactory.PLUGIN_NAME.equals(plugin.getInfo().name)) {
             pluginFactory.setDelegate(null);
           } else {
-            log.warn("Ignoring PlacementPluginFactory plugin with non-standard name: " + plugin.getInfo());
+            log.warn("Ignoring PlacementPluginFactory plugin with non-standard name: {}", plugin.getInfo());
           }
         }
       }
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 ae996ef..dbfa564 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
@@ -1,10 +1,25 @@
+/*
+ * 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.plugins;
 
 import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.util.ReflectMapWriter;
 
-import java.io.IOException;
-
 /**
  *
  */
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 b127263..f31dfa7 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -74,7 +74,6 @@ 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;
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 9cbdc3d..e048617 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
@@ -27,7 +27,6 @@ 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.PlacementRequestImpl;
-import org.apache.solr.cluster.placement.impl.PlacementRequestImpl;
 import org.apache.solr.common.util.Pair;
 import org.junit.BeforeClass;
 import org.junit.Test;

[lucene] 07/33: Move plugin Javadoc from private plugin to public factory

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 0cfcf837c4c9b881d671da65b0584358280c34c4
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Fri Nov 20 20:44:15 2020 +0100

    Move plugin Javadoc from private plugin to public factory
---
 .../plugins/AffinityPlacementFactory.java          | 94 +++++++++++-----------
 .../plugins/MinimizeCoresPlacementFactory.java     |  8 +-
 .../placement/plugins/RandomPlacementFactory.java  |  6 +-
 .../impl/AffinityPlacementFactoryTest.java         |  2 +-
 4 files changed, 54 insertions(+), 56 deletions(-)

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 db968e8..50ecb8d 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
@@ -67,6 +67,52 @@ import java.util.stream.Collectors;
     "set-placement-plugin" : null
   }' http://localhost:8983/api/cluster
  * </pre>
+ *
+ *
+ * <p>{@link AffinityPlacementPlugin} implements placing replicas in a way that replicate past Autoscaling config defined
+ * <a href="https://github.com/lucidworks/fusion-cloud-native/blob/master/policy.json#L16">here</a>.</p>
+ *
+ * <p>This specification is doing the following:
+ * <p><i>Spread replicas per shard as evenly as possible across multiple availability zones (given by a sys prop),
+ * assign replicas based on replica type to specific kinds of nodes (another sys prop), and avoid having more than
+ * one replica per shard on the same node.<br>
+ * Only after these constraints are satisfied do minimize cores per node or disk usage.</i></p>
+ *
+ * <p>Overall strategy of this plugin:</p>
+ * <ul><li>
+ *     The set of nodes in the cluster is obtained and transformed into 3 independent sets (that can overlap) of nodes
+ *     accepting each of the three replica types.
+ * </li><li>
+ *     For each shard on which placing replicas is required and then for each replica type to place (starting with NRT,
+ *     then TLOG then PULL): <ul>
+ *         <li>The set of candidates nodes corresponding to the replica type is used and from that set are removed nodes
+ *         that already have a replica (of any type) for that shard</li>
+ *         <li>If there are not enough nodes, an error is thrown (this is checked further down during processing).</li>
+ *         <li>The number of (already existing) replicas of the current type on each Availability Zone is collected.</li>
+ *         <li>Separate the set of available nodes to as many subsets (possibly some are empty) as there are Availability Zones
+ *         defined for the candidate nodes</li>
+ *         <li>In each AZ nodes subset, sort the nodes by increasing total number of cores count, with possibly a condition
+ *         that pushes nodes with low disk space to the end of the list? Or a weighted combination of the relative
+ *         importance of these two factors? Some randomization? Marking as non available nodes with not enough disk space?
+ *         These and other are likely aspects to be played with once the plugin is tested or observed to be running in prod,
+ *         don't expect the initial code drop(s) to do all of that.</li>
+ *         <li>Iterate over the number of replicas to place (for the current replica type for the current shard):
+ *         <ul>
+ *             <li>Based on the number of replicas per AZ collected previously, pick the non empty set of nodes having the
+ *             lowest number of replicas. Then pick the first node in that set. That's the node the replica is placed one.
+ *             Remove the node from the set of available nodes for the given AZ and increase the number of replicas placed
+ *             on that AZ.</li>
+ *         </ul></li>
+ *         <li>During this process, the number of cores on the nodes in general is tracked to take into account placement
+ *         decisions so that not all shards decide to put their replicas on the same nodes (they might though if these are
+ *         the less loaded nodes).</li>
+ *     </ul>
+ * </li>
+ * </ul>
+ *
+ * <p>This code is a realistic placement computation, based on a few assumptions. The code is written in such a way to
+ * 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 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -88,52 +134,8 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
   }
 
   /**
-   * <p>Implements placing replicas in a way that replicate past Autoscaling config defined
-   * <a href="https://github.com/lucidworks/fusion-cloud-native/blob/master/policy.json#L16">here</a>.</p>
-   *
-   * <p>This specification is doing the following:
-   * <p><i>Spread replicas per shard as evenly as possible across multiple availability zones (given by a sys prop),
-   * assign replicas based on replica type to specific kinds of nodes (another sys prop), and avoid having more than
-   * one replica per shard on the same node.<br>
-   * Only after these constraints are satisfied do minimize cores per node or disk usage.</i></p>
-   *
-   * <p>Overall strategy of this plugin:</p>
-   * <ul><li>
-   *     The set of nodes in the cluster is obtained and transformed into 3 independent sets (that can overlap) of nodes
-   *     accepting each of the three replica types.
-   * </li><li>
-   *     For each shard on which placing replicas is required and then for each replica type to place (starting with NRT,
-   *     then TLOG then PULL): <ul>
-   *         <li>The set of candidates nodes corresponding to the replica type is used and from that set are removed nodes
-   *         that already have a replica (of any type) for that shard</li>
-   *         <li>If there are not enough nodes, an error is thrown (this is checked further down during processing).</li>
-   *         <li>The number of (already existing) replicas of the current type on each Availability Zone is collected.</li>
-   *         <li>Separate the set of available nodes to as many subsets (possibly some are empty) as there are Availability Zones
-   *         defined for the candidate nodes</li>
-   *         <li>In each AZ nodes subset, sort the nodes by increasing total number of cores count, with possibly a condition
-   *         that pushes nodes with low disk space to the end of the list? Or a weighted combination of the relative
-   *         importance of these two factors? Some randomization? Marking as non available nodes with not enough disk space?
-   *         These and other are likely aspects to be played with once the plugin is tested or observed to be running in prod,
-   *         don't expect the initial code drop(s) to do all of that.</li>
-   *         <li>Iterate over the number of replicas to place (for the current replica type for the current shard):
-   *         <ul>
-   *             <li>Based on the number of replicas per AZ collected previously, pick the non empty set of nodes having the
-   *             lowest number of replicas. Then pick the first node in that set. That's the node the replica is placed one.
-   *             Remove the node from the set of available nodes for the given AZ and increase the number of replicas placed
-   *             on that AZ.</li>
-   *         </ul></li>
-   *         <li>During this process, the number of cores on the nodes in general is tracked to take into account placement
-   *         decisions so that not all shards decide to put their replicas on the same nodes (they might though if these are
-   *         the less loaded nodes).</li>
-   *     </ul>
-   * </li>
-   * </ul>
-   *
-   * <p>This code is a realistic placement computation, based on a few assumptions. The code is written in such a way to
-   * 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>
-   *
-   * See {@link AffinityPlacementFactory} for instructions on how to configure a cluster to use this plugin.
+   * See {@link AffinityPlacementFactory} for instructions on how to configure a cluster to use this plugin and details
+   * on what the plugin does.
    */
   static private class AffinityPlacementPlugin implements PlacementPlugin {
     /**
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 33d9c57..b73b692 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
@@ -34,7 +34,9 @@ import org.apache.solr.cluster.placement.*;
 import org.apache.solr.common.util.SuppressForbidden;
 
 /**
- * <p>Factory for creating {@link MinimizeCoresPlacementPlugin}.</p>
+ * <p>Factory for creating {@link MinimizeCoresPlacementPlugin}, a Placement plugin implementing placing replicas
+ * to minimize number of cores per {@link Node}, while not placing two replicas of the same shard on the same node.
+ * This code is meant as an educational example of a placement plugin.</p>
  *
  * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
  */
@@ -45,10 +47,6 @@ public class MinimizeCoresPlacementFactory implements PlacementPluginFactory {
     return new MinimizeCoresPlacementPlugin();
   }
 
-  /**
-   * Placement plugin implementing placing replicas to minimize number of cores per {@link Node}, while not placing two
-   * replicas of the same shard on the same node. This code is meant as an educational example of a placement plugin.
-   */
   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 7874c33..b9d3884 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
@@ -30,7 +30,8 @@ import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.*;
 
 /**
- * <p>Factory for creating {@link RandomPlacementPlugin}.</p>
+ * <p>Factory for creating {@link RandomPlacementPlugin}, a placement plugin implementing random placement for new
+ * collection creation while preventing two replicas of same shard from being placed on same node..</p>
  *
  * <p>See {@link AffinityPlacementFactory} for a more realistic example and documentation.</p>
  */
@@ -41,9 +42,6 @@ public class RandomPlacementFactory implements PlacementPluginFactory {
     return new RandomPlacementPlugin();
   }
 
-  /**
-   * Implements random placement for new collection creation while preventing two replicas of same shard from being placed on same node.
-   */
   static private class RandomPlacementPlugin implements PlacementPlugin {
 
     public PlacementPlan computePlacement(Cluster cluster, PlacementRequest request, AttributeFetcher attributeFetcher,
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
index 48c7eb6..80f30105 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
@@ -165,7 +165,7 @@ public class AffinityPlacementFactoryTest extends Assert {
         long start = System.nanoTime();
         PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
         long end = System.nanoTime();
-        log.info("ComputePlacement: {} nodes, {} shards, {} total replicas, elapsed time {} ms.", numNodes, numShards, TOTAL_REPLICAS, TimeUnit.NANOSECONDS.toMillis(end - start));
+        log.info("ComputePlacement: {} nodes, {} shards, {} total replicas, elapsed time {} ms.", numNodes, numShards, TOTAL_REPLICAS, TimeUnit.NANOSECONDS.toMillis(end - start)); //nowarn
         assertEquals("incorrect number of calculated placements", TOTAL_REPLICAS,
             pp.getReplicaPlacements().size());
         // check that replicas are correctly placed

[lucene] 19/33: Rename a couple of leftover deprioritizedFreeDiskGB into prioritizedFreeDiskGB

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 0a59e117a0dc8e1252fe00ec03b8fbacf7ec6d34
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Thu Nov 26 00:00:20 2020 +0100

    Rename a couple of leftover deprioritizedFreeDiskGB into prioritizedFreeDiskGB
---
 .../solr/cluster/placement/plugins/AffinityPlacementFactory.java      | 4 ++--
 .../solr/cluster/placement/plugins/AffinityPlacementFactoryTest.java  | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)

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 9690a86..0dd7340 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
@@ -45,7 +45,7 @@ import java.util.stream.Collectors;
     "set-placement-plugin": {
       "class": "org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory",
       "minimalFreeDiskGB": 10,
-      "deprioritizedFreeDiskGB": 50
+      "prioritizedFreeDiskGB": 50
     }
   }' http://localhost:8983/api/cluster
  * </pre>
@@ -57,7 +57,7 @@ import java.util.stream.Collectors;
  * "placement-plugin":{
  *     "class":"org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory",
  *     "minimalFreeDiskGB":10,
- *     "deprioritizedFreeDiskGB":50}
+ *     "prioritizedFreeDiskGB":50}
  * </pre>
  *
  * <p>In order to delete the placement-plugin section from {@code /clusterprops.json} (and to fallback to either Legacy
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 a5d71e7..3d3c1dc 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
@@ -51,7 +51,7 @@ public class AffinityPlacementFactoryTest extends SolrTestCaseJ4 {
   @BeforeClass
   public static void setupPlugin() {
     PlacementPluginConfig config = PlacementPluginConfigImpl.createConfigFromProperties(
-        Map.of("minimalFreeDiskGB", 10L, "deprioritizedFreeDiskGB", 50L));
+        Map.of("minimalFreeDiskGB", 10L, "prioritizedFreeDiskGB", 50L));
     plugin = new AffinityPlacementFactory().createPluginInstance(config);
     ((AffinityPlacementFactory.AffinityPlacementPlugin) plugin).setRandom(random());
   }

[lucene] 16/33: Remove PluginTestHelper, use Builders instead. Add getShardNames() to SolrCollection

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 2be41f3cecb285afad31ed9f359ddf5ea10f2eee
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Tue Nov 24 23:14:06 2020 +0100

    Remove PluginTestHelper, use Builders instead. Add getShardNames() to SolrCollection
---
 .../org/apache/solr/cluster/SolrCollection.java    |  33 ++++---
 .../impl/SimpleClusterAbstractionsImpl.java        |   5 +
 .../impl/AffinityPlacementFactoryTest.java         | 107 ++++++---------------
 .../solr/cluster/placement/impl/Builders.java      |   9 +-
 .../placement/impl/ClusterAbstractionsForTest.java |   9 +-
 .../cluster/placement/impl/PluginTestHelper.java   |  94 ------------------
 6 files changed, 62 insertions(+), 195 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cluster/SolrCollection.java b/solr/core/src/java/org/apache/solr/cluster/SolrCollection.java
index 23e79a4..d22560a 100644
--- a/solr/core/src/java/org/apache/solr/cluster/SolrCollection.java
+++ b/solr/core/src/java/org/apache/solr/cluster/SolrCollection.java
@@ -23,6 +23,7 @@ import org.apache.solr.cluster.placement.PlacementPlugin;
 import org.apache.solr.cluster.placement.PlacementRequest;
 
 import java.util.Iterator;
+import java.util.Set;
 
 /**
  * Represents a Collection in SolrCloud (unrelated to {@link java.util.Collection} that uses the nicer name).
@@ -55,20 +56,26 @@ public interface SolrCollection {
   Iterable<Shard> shards();
 
   /**
-   * <p>Returns the value of a custom property name set on the {@link SolrCollection} or {@code null} when no such
-   * property was set. Properties are set through the Collection API. See for example {@code COLLECTIONPROP} in the Solr reference guide.
-   *
-   * <p><b>{@link PlacementPlugin} related note:</b></p>
-   * <p>Using custom properties in conjunction with ad hoc {@link PlacementPlugin} code allows customizing placement
-   * decisions per collection.
-   *
-   * <p>For example if a collection is to be placed only on nodes using SSD storage and not rotating disks, it can be
-   * identified as such using some custom property (collection property could for example be called "driveType" and have
-   * value "ssd" in that case), and the placement plugin (implementing {@link PlacementPlugin}) would then
-   * {@link AttributeFetcher#requestNodeSystemProperty(String)} for that property from all nodes and only place replicas
-   * of this collection on {@link Node}'s for which
-   * {@link AttributeValues#getDiskType(Node)} is non empty and equal to {@link org.apache.solr.cluster.placement.AttributeFetcher.DiskHardwareType#SSD}.
+   * @return a set of the names of the shards defined for this collection. This set is backed by an internal map so should
+   * not be modified.
    */
+  Set<String> getShardNames();
+
+    /**
+     * <p>Returns the value of a custom property name set on the {@link SolrCollection} or {@code null} when no such
+     * property was set. Properties are set through the Collection API. See for example {@code COLLECTIONPROP} in the Solr reference guide.
+     *
+     * <p><b>{@link PlacementPlugin} related note:</b></p>
+     * <p>Using custom properties in conjunction with ad hoc {@link PlacementPlugin} code allows customizing placement
+     * decisions per collection.
+     *
+     * <p>For example if a collection is to be placed only on nodes using SSD storage and not rotating disks, it can be
+     * identified as such using some custom property (collection property could for example be called "driveType" and have
+     * value "ssd" in that case), and the placement plugin (implementing {@link PlacementPlugin}) would then
+     * {@link AttributeFetcher#requestNodeSystemProperty(String)} for that property from all nodes and only place replicas
+     * of this collection on {@link Node}'s for which
+     * {@link AttributeValues#getDiskType(Node)} is non empty and equal to {@link org.apache.solr.cluster.placement.AttributeFetcher.DiskHardwareType#SSD}.
+     */
   String getCustomProperty(String customPropertyName);
 
   /*
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java
index 6ea2d24..47b8f6c 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java
@@ -167,6 +167,11 @@ class SimpleClusterAbstractionsImpl {
     }
 
     @Override
+    public Set<String> getShardNames() {
+      return shards.keySet();
+    }
+
+    @Override
     public String getCustomProperty(String customPropertyName) {
       return docCollection.getStr(customPropertyName);
     }
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
index 3981b09..8f7e130 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
@@ -62,17 +62,12 @@ public class AffinityPlacementFactoryTest extends Assert {
     testBasicPlacementInternal(true);
   }
 
-  @Test
-  public void testBasicPlacementNewCollection2() throws Exception {
-    testBasicInternal2(false);
-  }
-
-  @Test
-  public void testBasicPlacementExistingCollection2() throws Exception {
-    testBasicInternal2(true);
-  }
-
-  private void testBasicInternal2(boolean hasExistingCollection) throws Exception {
+  /**
+   * When this test places a replica for a new collection, it should pick the node with less cores.<p>
+   * <p>
+   * When it places a replica for an existing collection, it should pick the node with more cores that doesn't already have a replica for the shard.
+   */
+  private void testBasicPlacementInternal(boolean hasExistingCollection) throws Exception {
     String collectionName = "testCollection";
 
     Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(2);
@@ -109,50 +104,6 @@ public class AffinityPlacementFactoryTest extends Assert {
     assertEquals(hasExistingCollection ? liveNodes.get(1) : liveNodes.get(0), rp.getNode());
   }
 
-  /**
-   * When this test places a replica for a new collection, it should pick the node with less cores.<p>
-   * <p>
-   * When it places a replica for an existing collection, it should pick the node with more cores that doesn't already have a replica for the shard.
-   */
-  private void testBasicPlacementInternal(boolean hasExistingCollection) throws Exception {
-    String collectionName = "testCollection";
-
-    Node node1 = new ClusterAbstractionsForTest.NodeImpl("node1");
-    Node node2 = new ClusterAbstractionsForTest.NodeImpl("node2");
-    Set<Node> liveNodes = Set.of(node1, node2);
-
-    ClusterAbstractionsForTest.SolrCollectionImpl solrCollection;
-    // Make sure new collections are not visible in the cluster state and existing ones are
-    final Map<String, SolrCollection> clusterCollections;
-    if (hasExistingCollection) {
-      // An existing collection with a single replica on node 1. Note that new collections already exist by the time the plugin is called, but are empty
-      solrCollection = PluginTestHelper.createCollection(collectionName, Map.of(), 1, 1, 0, 0, Set.of(node1));
-      clusterCollections = Map.of(solrCollection.getName(), solrCollection);
-    } else {
-      // A new collection has the shards defined ok but no replicas
-      solrCollection = PluginTestHelper.createCollection(collectionName, Map.of(), 1, 0, 0, 0, Set.of());
-      clusterCollections = Map.of();
-    }
-
-    Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, clusterCollections);
-    // Place a new replica for the (only) existing shard of the collection
-    PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, Set.of(solrCollection.shards().iterator().next().getShardName()), liveNodes, 1, 0, 0);
-    // More cores on node2
-    Map<Node, Integer> nodeToCoreCount = Map.of(node1, 1, node2, 10);
-    // A lot of free disk on the two nodes
-    final Map<Node, Long> nodeToFreeDisk = Map.of(node1, 100L, node2, 100L);
-    AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
-    AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
-    PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
-
-    PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
-
-
-    assertEquals(1, pp.getReplicaPlacements().size());
-    ReplicaPlacement rp = pp.getReplicaPlacements().iterator().next();
-    assertEquals(hasExistingCollection ? node2 : node1, rp.getNode());
-  }
-
   @Test
   public void testAvailabilityZones() throws Exception {
     String collectionName = "testCollection";
@@ -311,38 +262,34 @@ public class AffinityPlacementFactoryTest extends Assert {
                                   int nrtReplicas, int tlogReplicas,
                                   int pullReplicas) throws Exception {
 
-    int REPLICAS_PER_SHARD = nrtReplicas + tlogReplicas + pullReplicas;
-    int TOTAL_REPLICAS = numShards * REPLICAS_PER_SHARD;
-
-    String collectionName = "testCollection";
+    String collectionName = "scaleCollection";
 
-    final Set<Node> liveNodes = new HashSet<>();
-    final Map<Node, Long> nodeToFreeDisk = new HashMap<>();
-    final Map<Node, Integer> nodeToCoreCount = new HashMap<>();
+    Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(numNodes);
+    LinkedList<Builders.NodeBuilder> nodeBuilders = clusterBuilder.getNodeBuilders();
     for (int i = 0; i < numNodes; i++) {
-      Node node = new ClusterAbstractionsForTest.NodeImpl("node_" + i);
-      liveNodes.add(node);
-      nodeToFreeDisk.put(node, Long.valueOf(numNodes));
-      nodeToCoreCount.put(node, 0);
+      nodeBuilders.get(i).setCoreCount(0).setFreeDiskGB(Long.valueOf(numNodes));
     }
-    ClusterAbstractionsForTest.SolrCollectionImpl solrCollection =
-        PluginTestHelper.createCollection(collectionName, Map.of(), numShards, 0, 0, 0, Set.of());
 
-    Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, Map.of());
-    PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
-        // XXX awkward!
-        // StreamSupport.stream(solrCollection.shards().spliterator(), false)
-        //     .map(Shard::getShardName).collect(Collectors.toSet()),
-        solrCollection.getShardNames(),
-        liveNodes, nrtReplicas, tlogReplicas, pullReplicas);
-
-    AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
-    AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
-    PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+    Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
+    collectionBuilder.initializeShardsReplicas(numShards, 0, 0, 0, List.of());
+
+    Cluster cluster = clusterBuilder.build();
+    AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
+
+    SolrCollection solrCollection = collectionBuilder.build();
+    List<Node> liveNodes = clusterBuilder.buildLiveNodes();
+
+    // Place replicas for all the shards of the (newly created since it has no replicas yet) collection
+    PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, solrCollection.getShardNames(),
+        new HashSet<>(liveNodes), nrtReplicas, tlogReplicas, pullReplicas);
 
     long start = System.nanoTime();
-    PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+    PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, new PlacementPlanFactoryImpl());
     long end = System.nanoTime();
+
+    final int REPLICAS_PER_SHARD = nrtReplicas + tlogReplicas + pullReplicas;
+    final int TOTAL_REPLICAS = numShards * REPLICAS_PER_SHARD;
+
     log.info("ComputePlacement: {} nodes, {} shards, {} total replicas, elapsed time {} ms.", numNodes, numShards, TOTAL_REPLICAS, TimeUnit.NANOSECONDS.toMillis(end - start)); //nowarn
     assertEquals("incorrect number of calculated placements", TOTAL_REPLICAS,
         pp.getReplicaPlacements().size());
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
index 1d964b2..1880374 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
@@ -27,7 +27,7 @@ public class Builders {
     ClusterBuilder initializeNodes(int countNodes) {
       nodeBuilders = new LinkedList<>();
       for (int n = 0; n < countNodes; n++) {
-        nodeBuilders.add(new NodeBuilder().setNodeName("node" + n)); // Default name, can be changed
+        nodeBuilders.add(new NodeBuilder().setNodeName("node_" + n)); // Default name, can be changed
       }
       return this;
     }
@@ -128,9 +128,10 @@ public class Builders {
       Iterator<NodeBuilder> nodeIterator = nodes.iterator();
 
       shardBuilders = new LinkedList<>();
+      int replicaNumber = 0;
 
-      for (int s = 0; s < countShards; s++) {
-        String shardName = "shard" + (s + 1);
+      for (int shardNumber = 1; shardNumber <= countShards; shardNumber++) {
+        String shardName = "shard" + shardNumber;
 
         LinkedList<ReplicaBuilder> replicas = new LinkedList<>();
         ReplicaBuilder leader = null;
@@ -146,7 +147,7 @@ public class Builders {
           int count = tc.second();
           String replicaPrefix = collectionName + "_" + shardName + "_replica_" + type.getSuffixChar();
           for (int r = 0; r < count; r++) {
-            String replicaName = replicaPrefix + r;
+            String replicaName = replicaPrefix + replicaNumber++;
             String coreName = replicaName + "_c";
             if (!nodeIterator.hasNext()) {
               nodeIterator = nodes.iterator();
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
index bd14d0d..059143b 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
@@ -156,10 +156,6 @@ class ClusterAbstractionsForTest {
       this.shards = shards;
     }
 
-    Set<String> getShardNames() {
-      return shards.keySet();
-    }
-
     @Override
     public String getName() {
       return collectionName;
@@ -182,6 +178,11 @@ class ClusterAbstractionsForTest {
     }
 
     @Override
+    public Set<String> getShardNames() {
+      return shards.keySet();
+    }
+
+    @Override
     public String getCustomProperty(String customPropertyName) {
       return customProperties.get(customPropertyName);
     }
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
deleted file mode 100644
index 61670da..0000000
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
+++ /dev/null
@@ -1,94 +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 org.apache.solr.cluster.Node;
-import org.apache.solr.cluster.Replica;
-import org.apache.solr.cluster.Shard;
-import org.apache.solr.cluster.SolrCollection;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-
-public class PluginTestHelper {
-
-  static ClusterAbstractionsForTest.SolrCollectionImpl createCollection(String name, Map<String, String> properties,
-                                                                        int numShards, int nrtReplicas, int tlogReplicas, int pullReplicas, Set<Node> nodes) {
-    ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(name, properties);
-    Map<String, Shard> shards = createShardsAndReplicas(solrCollection, numShards, nrtReplicas, tlogReplicas, pullReplicas, nodes);
-    solrCollection.setShards(shards);
-    return solrCollection;
-  }
-
-  /**
-   * Builds the representation of shards for a collection, based on the number of shards and replicas for each to create.
-   * The replicas are allocated to the provided nodes in a round robin way. The leader is set to the last replica of each shard.
-   */
-  static Map<String, Shard> createShardsAndReplicas(SolrCollection collection, int numShards,
-                                                    int nrtReplicas, int tlogReplicas, int pullReplicas,
-                                                    Set<Node> nodes) {
-    Iterator<Node> nodeIterator = nodes.iterator();
-
-    Map<String, Shard> shards = new HashMap<>();
-
-    for (int s = 0; s < numShards; s++) {
-      // "traditional" shard name
-      String shardName = "shard" + (s + 1);
-
-      ClusterAbstractionsForTest.ShardImpl shard = new ClusterAbstractionsForTest.ShardImpl(shardName, collection, Shard.ShardState.ACTIVE);
-
-      Map<String, Replica> replicas = new HashMap<>();
-
-      Replica leader = null;
-      int totalReplicas = nrtReplicas + tlogReplicas + pullReplicas;
-      for (int r = 0; r < totalReplicas; r++) {
-        Replica.ReplicaType type;
-        if (r < nrtReplicas) {
-          type = Replica.ReplicaType.NRT;
-        } else if (r < nrtReplicas + tlogReplicas) {
-          type = Replica.ReplicaType.TLOG;
-        } else {
-          type = Replica.ReplicaType.PULL;
-        }
-        String replicaName = shardName + "_replica_" + type.getSuffixChar() + r;
-        String coreName = replicaName + "_c";
-        final Node node;
-        if (!nodeIterator.hasNext()) {
-          nodeIterator = nodes.iterator();
-        }
-        // If the nodes set is empty, this call will fail
-        node = nodeIterator.next();
-
-        Replica replica = new ClusterAbstractionsForTest.ReplicaImpl(replicaName, coreName, shard, type, Replica.ReplicaState.ACTIVE, node);
-
-        replicas.put(replica.getReplicaName(), replica);
-        if (replica.getType() == Replica.ReplicaType.NRT) {
-          leader = replica;
-        }
-      }
-
-      shard.setReplicas(replicas, leader);
-
-      shards.put(shard.getShardName(), shard);
-    }
-
-    return shards;
-  }
-}

[lucene] 12/33: SOLR-15004: Fix the use of prefixes for sysprops and metrics. Finish the affinity test.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 20a2778019cfc9999fafab7f5035284d8d5b30c1
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue Nov 24 15:18:26 2020 +0100

    SOLR-15004: Fix the use of prefixes for sysprops and metrics. Finish the affinity test.
---
 .../impl/AffinityPlacementFactoryTest.java         | 41 ++++++++++++++++------
 .../solr/cluster/placement/impl/Builders.java      |  8 +++--
 2 files changed, 35 insertions(+), 14 deletions(-)

diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
index 6b64864..67350d0 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
@@ -183,19 +183,38 @@ public class AffinityPlacementFactoryTest extends Assert {
             cluster.getLiveNodes(), 2, 2, 2);
 
         PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
-
-        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, clusterBuilder.buildAttributeFetcher(), placementPlanFactory);
+        AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
+        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
         // 2 shards, 6 replicas
         assertEquals(12, pp.getReplicaPlacements().size());
-        List<ReplicaPlacement> placements = new ArrayList<>(pp.getReplicaPlacements());
-        Collections.sort(placements, Comparator
-            .comparing((ReplicaPlacement p) -> p.getNode().getName())
-            .thenComparing((ReplicaPlacement p) -> p.getShardName())
-            .thenComparing((ReplicaPlacement p) -> p.getReplicaType())
-        );
-        log.info(placements.toString());  //nowarn
-        // AZ -> shard -> replica count
-        //Map<String, Map<String, AtomicInteger>>
+//        List<ReplicaPlacement> placements = new ArrayList<>(pp.getReplicaPlacements());
+//        Collections.sort(placements, Comparator
+//            .comparing((ReplicaPlacement p) -> p.getNode().getName())
+//            .thenComparing((ReplicaPlacement p) -> p.getShardName())
+//            .thenComparing((ReplicaPlacement p) -> p.getReplicaType())
+//        );
+        // shard -> AZ -> replica count
+        Map<Replica.ReplicaType, Map<String, Map<String, AtomicInteger>>> replicas = new HashMap<>();
+        AttributeValues attributeValues = attributeFetcher.fetchAttributes();
+        for (ReplicaPlacement rp : pp.getReplicaPlacements()) {
+            Optional<String> azOptional = attributeValues.getSystemProperty(rp.getNode(), AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP);
+            if (!azOptional.isPresent()) {
+                fail("missing AZ sysprop for node " + rp.getNode());
+            }
+            String az = azOptional.get();
+            replicas.computeIfAbsent(rp.getReplicaType(), type -> new HashMap<>())
+                .computeIfAbsent(rp.getShardName(), shard -> new HashMap<>())
+                .computeIfAbsent(az, zone -> new AtomicInteger()).incrementAndGet();
+        }
+        replicas.forEach((type, perTypeReplicas) -> {
+            perTypeReplicas.forEach((shard, azCounts) -> {
+                assertEquals("number of AZs", 2, azCounts.size());
+                azCounts.forEach((az, count) -> {
+                    assertTrue("too few replicas shard=" + shard + ", type=" + type + ", az=" + az,
+                        count.get() >= 1);
+                });
+            });
+        });
     }
 
     @Test
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
index 996186f..aa5092b 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
@@ -291,15 +291,17 @@ public class Builders {
             if (sysprops == null) {
                 sysprops = new HashMap<>();
             }
-            sysprops.put(key, value);
+            String name = AttributeFetcherImpl.getSystemPropertySnitchTag(key);
+            sysprops.put(name, value);
             return this;
         }
 
-        NodeBuilder setMetric(String key, Double value) {
+        NodeBuilder setMetric(AttributeFetcher.NodeMetricRegistry registry, String key, Double value) {
             if (metrics == null) {
                 metrics = new HashMap<>();
             }
-            metrics.put(key, value);
+            String name = AttributeFetcherImpl.getMetricSnitchTag(key, registry);
+            metrics.put(name, value);
             return this;
         }
 

[lucene] 30/33: SOLR-15016: More fixes after review.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 60c8fb3ef909a22e6915262f3337a2c761705cb2
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Fri Dec 4 21:49:13 2020 +0100

    SOLR-15016: More fixes after review.
---
 .../org/apache/solr/api/ConfigurablePlugin.java    |  4 ++--
 .../cluster/placement/PlacementPluginFactory.java  | 22 ++++++++++++++++----
 .../impl/DelegatingPlacementPluginFactory.java     | 10 ---------
 .../impl/PlacementPluginFactoryLoader.java         |  6 +++---
 .../plugins/MinimizeCoresPlacementFactory.java     | 10 ---------
 .../placement/plugins/RandomPlacementFactory.java  | 24 ++++++++--------------
 .../OverseerCollectionConfigSetProcessorTest.java  |  5 +++--
 7 files changed, 34 insertions(+), 47 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/api/ConfigurablePlugin.java b/solr/core/src/java/org/apache/solr/api/ConfigurablePlugin.java
index ef13d8a..0d9a183 100644
--- a/solr/core/src/java/org/apache/solr/api/ConfigurablePlugin.java
+++ b/solr/core/src/java/org/apache/solr/api/ConfigurablePlugin.java
@@ -25,8 +25,8 @@ import org.apache.solr.common.MapWriter;
  */
 public interface ConfigurablePlugin<T extends MapWriter> {
 
-  /**This is invoked soon after the Object is initialized
-   * 
+  /**
+   * This is invoked soon after the Object is initialized.
    * @param cfg value deserialized from JSON
    */
   void configure(T cfg);
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 1358a93..3bfc0d8 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
@@ -20,7 +20,9 @@ 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
+ * Factory implemented by client code and configured in container plugins
+ * (see {@link org.apache.solr.handler.admin.ContainerPluginsApi#editAPI})
+ * allowing the creation of instances of
  * {@link PlacementPlugin} to be used for replica placement computation.
  * <p>Note: configurable factory implementations should also implement
  * {@link org.apache.solr.api.ConfigurablePlugin} with the appropriate configuration
@@ -42,14 +44,26 @@ public interface PlacementPluginFactory<T extends PlacementPluginConfig> extends
   PlacementPlugin createPluginInstance();
 
   /**
+   * Default implementation is a no-op. Override to provide meaningful
+   * behavior if needed.
+   * @param cfg value deserialized from JSON, not null.
+   */
+  @Override
+  default void configure(T cfg) {
+    // no-op
+  }
+
+  /**
    * Return the configuration of the plugin.
+   * Default implementation returns null.
    */
-  T getConfig();
+  default T getConfig() {
+    return null;
+  }
 
   /**
-   * Useful for plugins that don't use any configuration.
+   * Useful type 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
index b8c5214..9786fd5 100644
--- 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
@@ -50,14 +50,4 @@ public final class DelegatingPlacementPluginFactory implements PlacementPluginFa
   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 4348a0d..be534b3 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
@@ -46,7 +46,7 @@ public class PlacementPluginFactoryLoader {
         }
         Object instance = plugin.getInstance();
         if (instance instanceof PlacementPluginFactory) {
-          setDelegate(plugin.getInfo(), instance);
+          setDelegate(plugin.getInfo(), (PlacementPluginFactory<? extends PlacementPluginConfig>) instance);
         }
       }
 
@@ -66,9 +66,9 @@ public class PlacementPluginFactoryLoader {
         added(replacement);
       }
 
-      private void setDelegate(PluginMeta pluginMeta, Object instance) {
+      private void setDelegate(PluginMeta pluginMeta, PlacementPluginFactory<? extends PlacementPluginConfig> factory) {
         if (PlacementPluginFactory.PLUGIN_NAME.equals(pluginMeta.name)) {
-          pluginFactory.setDelegate((PlacementPluginFactory<? extends PlacementPluginConfig>) instance);
+          pluginFactory.setDelegate(factory);
         } else {
           log.warn("Ignoring PlacementPluginFactory plugin with non-standard name: {}", pluginMeta);
         }
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 f0a2b6e..5038ddd 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
@@ -47,16 +47,6 @@ public class MinimizeCoresPlacementFactory implements PlacementPluginFactory<Pla
     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 9829e6c..0b27d21 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
@@ -23,7 +23,6 @@ import java.util.HashSet;
 import java.util.Random;
 import java.util.Set;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.solr.cluster.Cluster;
 import org.apache.solr.cluster.Node;
 import org.apache.solr.cluster.Replica;
@@ -43,22 +42,15 @@ public class RandomPlacementFactory implements PlacementPluginFactory<PlacementP
     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();
+    private final Random replicaPlacementRandom = new Random(); // ok even if random sequence is predictable.
 
-    @VisibleForTesting
-    public void setRandom(Random random) {
-      this.random = random;
+    private RandomPlacementPlugin() {
+      // We make things reproducible in tests by using test seed if any
+      String seed = System.getProperty("tests.seed");
+      if (seed != null) {
+        replicaPlacementRandom.setSeed(seed.hashCode());
+      }
     }
 
     public PlacementPlan computePlacement(Cluster cluster, PlacementRequest request, AttributeFetcher attributeFetcher,
@@ -78,7 +70,7 @@ public class RandomPlacementFactory implements PlacementPluginFactory<PlacementP
       for (String shardName : request.getShardNames()) {
         // Shuffle the nodes for each shard so that replicas for a shard are placed on distinct yet random nodes
         ArrayList<Node> nodesToAssign = new ArrayList<>(cluster.getLiveNodes());
-        Collections.shuffle(nodesToAssign, random);
+        Collections.shuffle(nodesToAssign, replicaPlacementRandom);
 
         for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
           placeForReplicaType(request.getCollection(), nodesToAssign, placementPlanFactory, replicaPlacements, shardName, request, rt);
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 694ad4a..ee5caeb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
@@ -43,7 +43,6 @@ 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;
@@ -123,6 +122,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
   private static CoreContainer coreContainerMock;
   private static UpdateShardHandler updateShardHandlerMock;
   private static HttpClient httpClientMock;
+  @SuppressWarnings("rawtypes")
   private static PlacementPluginFactory placementPluginFactoryMock;
   
   private static ObjectCache objectCache;
@@ -250,7 +250,8 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
     stopComponentUnderTest();
     super.tearDown();
   }
-  
+
+  @SuppressWarnings("unchecked")
   protected Set<String> commonMocks(int liveNodesCount) throws Exception {
     when(shardHandlerFactoryMock.getShardHandler()).thenReturn(shardHandlerMock);
     when(workQueueMock.peekTopN(anyInt(), any(), anyLong())).thenAnswer(invocation -> {

[lucene] 22/33: SOLR-15016: Remove unneeded file.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 29fd3f978733efab451adcdafe28db54bf7e7c29
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Fri Nov 27 12:27:50 2020 +0100

    SOLR-15016: Remove unneeded file.
---
 .../cluster/placement/PlacementPluginConfig.java   | 116 ---------------------
 .../cluster/placement/PlacementPluginFactory.java  |   5 +-
 2 files changed, 4 insertions(+), 117 deletions(-)

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
deleted file mode 100644
index 1518377..0000000
--- a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPluginConfig.java
+++ /dev/null
@@ -1,116 +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;
-
-/**
- * <p>Configuration passed by Solr to {@link PlacementPluginFactory#createPluginInstance(PlacementPluginConfig)} so that plugin instances
- * ({@link PlacementPlugin}) created by the factory can easily retrieve their configuration.</p>
- *
- * <p>A plugin writer decides the names and the types of the configurable parameters it needs. Available types are
- * {@link String}, {@link Long}, {@link Boolean}, {@link Double}. This configuration currently lives in the {@code /clusterprops.json}
- * file in Zookeeper (this could change in the future, the plugin code will not change but the way to store its configuration
- * in the cluster might). {@code clusterprops.json} also contains the name of the plugin factory class implementing
- * {@link org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory}.</p>
- *
- * <p>In order to configure a plugin to be used for placement decisions, the following {@code curl} command (or something
- * equivalent) has to be executed once the cluster is already running to set the configuration.
- * Replace {@code localhost:8983} by one of your servers' IP address and port.</p>
- *
- * <pre>
- *
- * curl -X POST -H 'Content-type:application/json' -d '{
- *   "set-placement-plugin": {
- *     "class": "factory.class.name$inner",
- *     "myfirstString": "a text value",
- *     "aLong": 50,
- *     "aDoubleConfig": 3.1415928,
- *     "shouldIStay": true
- *   }
- * }' http://localhost:8983/api/cluster
- * </pre>
- *
- * <p>The consequence will be the creation (or replacement if it exists) of an element in the Zookeeper file
- * {@code /clusterprops.json} as follows:</p>
- *
- * <pre>
- *
- * "placement-plugin":{
- *     "class":"factory.class.name$inner",
- *     "myfirstString": "a text value",
- *     "aLong": 50,
- *     "aDoubleConfig": 3.1415928,
- *     "shouldIStay": true}
- * </pre>
- *
- * <p>In order to delete the placement-plugin section from {@code /clusterprops.json} (and to fallback to either Legacy
- * or rule based placement if so configured for a collection), execute:</p>
- *
- * <pre>
- *
- * curl -X POST -H 'Content-type:application/json' -d '{
- *   "set-placement-plugin" : null
- * }' http://localhost:8983/api/cluster
- * </pre>
- */
-public interface PlacementPluginConfig {
-
-  /**
-   * @return the configured {@link String} value corresponding to {@code configName} if one exists (could be the empty
-   * string) and {@code null} otherwise.
-   */
-  String getStringConfig(String configName);
-
-  /**
-   * @return the configured {@link String} value corresponding to {@code configName} if one exists (could be the empty
-   * string) and {@code defaultValue} otherwise.
-   */
-  String getStringConfig(String configName, String defaultValue);
-
-  /**
-   * @return the configured {@link Boolean} value corresponding to {@code configName} if one exists, {@code null} otherwise.
-   */
-  Boolean getBooleanConfig(String configName);
-
-  /**
-   * @return the configured {@link Boolean} value corresponding to {@code configName} if one exists, a boxed {@code defaultValue}
-   * otherwise (this method never returns {@code null}.
-   */
-  Boolean getBooleanConfig(String configName, boolean defaultValue);
-
-  /**
-   * @return the configured {@link Long} value corresponding to {@code configName} if one exists, {@code null} otherwise.
-   */
-  Long getLongConfig(String configName);
-
-  /**
-   * @return the configured {@link Long} value corresponding to {@code configName} if one exists, a boxed {@code defaultValue}
-   * otherwise (this method never returns {@code null}.
-   */
-  Long getLongConfig(String configName, long defaultValue);
-
-  /**
-   * @return the configured {@link Double} value corresponding to {@code configName} if one exists, {@code null} otherwise.
-   */
-  Double getDoubleConfig(String configName);
-
-  /**
-   * @return the configured {@link Double} value corresponding to {@code configName} if one exists, a boxed {@code defaultValue}
-   * otherwise (this method never returns {@code null}.
-   */
-  Double getDoubleConfig(String configName, double defaultValue);
-}
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 7df4d86..abdd7b9 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
@@ -18,8 +18,11 @@
 package org.apache.solr.cluster.placement;
 
 /**
- * Factory implemented by client code and configured in {@code solr.xml} allowing the creation of instances of
+ * 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.
+ * <p>Note: configurable factory implementations should also implement
+ * {@link org.apache.solr.api.ConfigurablePlugin} with the appropriate configuration
+ * bean type.</p>
  */
 public interface PlacementPluginFactory {
   /**

[lucene] 23/33: SOLR-15016: Add unit test for dynamic reconfiguration. Remove old API endpoint.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 68536407cee0e2f4049704243071a916251c9453
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Fri Nov 27 13:19:56 2020 +0100

    SOLR-15016: Add unit test for dynamic reconfiguration. Remove old API endpoint.
---
 .../solr/cluster/placement/PlacementPlugin.java    |   5 -
 .../impl/PlacementPluginFactoryLoader.java         |  13 ++-
 .../plugins/AffinityPlacementFactory.java          |   5 +
 .../java/org/apache/solr/handler/ClusterAPI.java   |  23 ----
 .../impl/PlacementPluginIntegrationTest.java       | 122 +++++++++++++++++++--
 5 files changed, 129 insertions(+), 39 deletions(-)

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 11b08dd..bbb52cb 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,11 +31,6 @@ 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/impl/PlacementPluginFactoryLoader.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginFactoryLoader.java
index f444834..8207279 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
@@ -52,9 +52,11 @@ public class PlacementPluginFactoryLoader {
   /**
    * Helper class to support dynamic reloading of plugin implementations.
    */
-  private static final class DelegatingPlacementPluginFactory implements PlacementPluginFactory {
+  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() {
@@ -67,6 +69,15 @@ public class PlacementPluginFactoryLoader {
 
     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/AffinityPlacementFactory.java b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java
index 4867ab1..a4faa7e 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
@@ -178,6 +178,11 @@ public class AffinityPlacementFactory implements PlacementPluginFactory, Configu
     this.config = cfg;
   }
 
+  @VisibleForTesting
+  public AffinityPlacementConfig getConfig() {
+    return config;
+  }
+
   /**
    * See {@link AffinityPlacementFactory} for instructions on how to configure a cluster to use this plugin and details
    * on what the plugin does.
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 157c0e5..ee77e3d 100644
--- a/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/ClusterAPI.java
@@ -27,9 +27,6 @@ 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.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;
 import org.apache.solr.common.cloud.ClusterProperties;
@@ -244,26 +241,6 @@ public class ClusterAPI {
       collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), m), obj.getResponse());
     }
 
-    @Command(name = "set-placement-plugin")
-    public void setPlacementPlugin(PayloadObj<Map<String, Object>> obj) {
-      Map<String, Object> placementPluginConfig = obj.getDataMap();
-      if(placementPluginConfig.isEmpty()) placementPluginConfig = null;
-      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(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),
-            PlacementPluginFactory.PLUGIN_NAME);
-      } catch (Exception e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error in API", e);
-      }
-    }
-
     @Command(name = "set-ratelimiter")
     public void setRateLimiters(PayloadObj<RateLimiterMeta> payLoad) {
       RateLimiterMeta rateLimiterConfig = payLoad.get();
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 90e3965..8c218e7 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
@@ -3,19 +3,29 @@ package org.apache.solr.cluster.placement.impl;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.V2Request;
+import org.apache.solr.client.solrj.request.beans.PluginMeta;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.V2Response;
 import org.apache.solr.cloud.SolrCloudTestCase;
-import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementPluginFactory;
+import org.apache.solr.cluster.placement.plugins.AffinityPlacementConfig;
+import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
 import org.apache.solr.cluster.placement.plugins.MinimizeCoresPlacementFactory;
 import org.apache.solr.common.cloud.ClusterProperties;
 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.TimeOut;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static java.util.Collections.singletonMap;
@@ -29,6 +39,7 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
 
   private static ClusterProperties clusterProperties;
   private static SolrCloudManager cloudManager;
+  private static CoreContainer cc;
 
   @BeforeClass
   public static void setupCluster() throws Exception {
@@ -37,29 +48,38 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
     configureCluster(3)
         .addConfig("conf", configset("cloud-minimal"))
         .configure();
-    cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
+    cc = cluster.getJettySolrRunner(0).getCoreContainer();
+    cloudManager = cc.getZkController().getSolrCloudManager();
     clusterProperties = new ClusterProperties(cluster.getZkClient());
   }
 
   @After
   public void cleanup() throws Exception {
     cluster.deleteAllCollections();
-    V2Request req = new V2Request.Builder("/cluster")
+    V2Request req = new V2Request.Builder("/cluster/plugin")
         .forceV2(true)
-        .POST()
-        .withPayload(singletonMap("set-placement-plugin", Map.of()))
+        .GET()
         .build();
-    req.process(cluster.getSolrClient());
-
+    V2Response rsp = req.process(cluster.getSolrClient());
+    if (rsp._get(Arrays.asList("plugin", PlacementPluginFactory.PLUGIN_NAME), null) != null) {
+      req = new V2Request.Builder("/cluster/plugin")
+          .forceV2(true)
+          .POST()
+          .withPayload("{remove: " + PlacementPluginFactory.PLUGIN_NAME + "}")
+          .build();
+      req.process(cluster.getSolrClient());
+    }
   }
 
   @Test
   public void testMinimizeCores() throws Exception {
-    Map<String, Object> config = Map.of(PlacementPlugin.FACTORY_CLASS, MinimizeCoresPlacementFactory.class.getName());
-    V2Request req = new V2Request.Builder("/cluster")
+    PluginMeta plugin = new PluginMeta();
+    plugin.name = PlacementPluginFactory.PLUGIN_NAME;
+    plugin.klass = MinimizeCoresPlacementFactory.class.getName();
+    V2Request req = new V2Request.Builder("/cluster/plugin")
         .forceV2(true)
         .POST()
-        .withPayload(singletonMap("set-placement-plugin", config))
+        .withPayload(singletonMap("add", plugin))
         .build();
     req.process(cluster.getSolrClient());
 
@@ -91,4 +111,86 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
     assertEquals("min cores too low", 1, minCores);
   }
 
+  @Test
+  public void testDynamicReconfiguration() throws Exception {
+    PluginMeta plugin = new PluginMeta();
+    plugin.name = PlacementPluginFactory.PLUGIN_NAME;
+    plugin.klass = MinimizeCoresPlacementFactory.class.getName();
+    V2Request req = new V2Request.Builder("/cluster/plugin")
+        .forceV2(true)
+        .POST()
+        .withPayload(singletonMap("add", plugin))
+        .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();
+    assertTrue("wrong version " + version, version > 0);
+    PlacementPluginFactory factory = wrapper.getDelegate();
+    assertTrue("wrong type " + factory.getClass().getName(), factory instanceof MinimizeCoresPlacementFactory);
+
+    // reconfigure
+    plugin.klass = AffinityPlacementFactory.class.getName();
+    plugin.config = new AffinityPlacementConfig(1, 2);
+    req = new V2Request.Builder("/cluster/plugin")
+        .forceV2(true)
+        .POST()
+        .withPayload(singletonMap("update", plugin))
+        .build();
+    req.process(cluster.getSolrClient());
+
+    version = waitForVersionChange(version, wrapper);
+
+    factory = wrapper.getDelegate();
+    assertTrue("wrong type " + factory.getClass().getName(), factory instanceof AffinityPlacementFactory);
+    AffinityPlacementConfig config = ((AffinityPlacementFactory) factory).getConfig();
+    assertEquals("minimalFreeDiskGB", 1, config.minimalFreeDiskGB);
+    assertEquals("prioritizedFreeDiskGB", 2, config.prioritizedFreeDiskGB);
+
+    // change plugin config
+    plugin.config = new AffinityPlacementConfig(3, 4);
+    req = new V2Request.Builder("/cluster/plugin")
+        .forceV2(true)
+        .POST()
+        .withPayload(singletonMap("update", plugin))
+        .build();
+    req.process(cluster.getSolrClient());
+
+    version = waitForVersionChange(version, wrapper);
+    factory = wrapper.getDelegate();
+    assertTrue("wrong type " + factory.getClass().getName(), factory instanceof AffinityPlacementFactory);
+    config = ((AffinityPlacementFactory) factory).getConfig();
+    assertEquals("minimalFreeDiskGB", 3, config.minimalFreeDiskGB);
+    assertEquals("prioritizedFreeDiskGB", 4, config.prioritizedFreeDiskGB);
+
+    // remove plugin
+    req = new V2Request.Builder("/cluster/plugin")
+        .forceV2(true)
+        .POST()
+        .withPayload("{remove: " + PlacementPluginFactory.PLUGIN_NAME + "}")
+        .build();
+    req.process(cluster.getSolrClient());
+    version = waitForVersionChange(version, wrapper);
+    factory = wrapper.getDelegate();
+    assertNull("no factory should be present", factory);
+  }
+
+  private int waitForVersionChange(int currentVersion, PlacementPluginFactoryLoader.DelegatingPlacementPluginFactory wrapper) throws Exception {
+    TimeOut timeout = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+
+    while (!timeout.hasTimedOut()) {
+      int newVersion = wrapper.getVersion();
+      if (newVersion < currentVersion) {
+        throw new Exception("Invalid version - went back! currentVersion=" + currentVersion +
+            " newVersion=" + newVersion);
+      } else if (currentVersion < newVersion) {
+        return newVersion;
+      }
+      timeout.sleep(200);
+    }
+    throw new TimeoutException("version didn't change in time, currentVersion=" + currentVersion);
+  }
 }

[lucene] 09/33: Introduce a builder class for unit tests to simplify creating cluster states and collections

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit bd63ad16e5ab73ab2c794b2a1f01fe75fbf42a95
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Tue Nov 24 01:57:29 2020 +0100

    Introduce a builder class for unit tests to simplify creating cluster states and collections
---
 .../impl/AffinityPlacementFactoryTest.java         |  52 +++-
 .../solr/cluster/placement/impl/Builders.java      | 287 +++++++++++++++++++++
 .../placement/impl/ClusterAbstractionsForTest.java |   4 +-
 3 files changed, 337 insertions(+), 6 deletions(-)

diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
index 80f30105..bf62c17 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
@@ -31,10 +31,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.lang.invoke.MethodHandles;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -63,6 +60,53 @@ public class AffinityPlacementFactoryTest extends Assert {
         testBasicPlacementInternal(true);
     }
 
+    @Test
+    public void testBasicPlacementNewCollection2() throws Exception {
+        testBasicInternal2(false);
+    }
+
+    @Test
+    public void testBasicPlacementExistingCollection2() throws Exception {
+        testBasicInternal2(true);
+    }
+
+    private void testBasicInternal2(boolean hasExistingCollection) throws Exception {
+        String collectionName = "testCollection";
+
+        Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(2);
+        LinkedList<Builders.NodeBuilder> nodeBuilders = clusterBuilder.getNodeBuilders();
+        nodeBuilders.get(0).setCoreCount(1).setFreeDiskGB(100L);
+        nodeBuilders.get(1).setCoreCount(10).setFreeDiskGB(100L);
+
+        Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
+
+        if (hasExistingCollection) {
+            // Existing collection has replicas for its shards and is visible in the cluster state
+            collectionBuilder.initializeShardsReplicas(1, 1, 0, 0, nodeBuilders);
+            clusterBuilder.addCollection(collectionBuilder);
+        } else {
+            // New collection to create has the shards defined but no replicas and is not present in cluster state
+            collectionBuilder.initializeShardsReplicas(1, 0, 0, 0, List.of());
+        }
+
+        Cluster cluster = clusterBuilder.build();
+        AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
+
+        SolrCollection solrCollection = collectionBuilder.build();
+        List<Node> liveNodes = clusterBuilder.buildLiveNodes();
+
+        // Place a new replica for the (only) existing shard of the collection
+        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
+                Set.of(solrCollection.shards().iterator().next().getShardName()), new HashSet<>(liveNodes),
+                1, 0, 0);
+
+        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, new PlacementPlanFactoryImpl());
+
+        assertEquals(1, pp.getReplicaPlacements().size());
+        ReplicaPlacement rp = pp.getReplicaPlacements().iterator().next();
+        assertEquals(hasExistingCollection ? liveNodes.get(1) : liveNodes.get(0), rp.getNode());
+    }
+
     /**
      * When this test places a replica for a new collection, it should pick the node with less cores.<p>
      *
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
new file mode 100644
index 0000000..d954dd1
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
@@ -0,0 +1,287 @@
+package org.apache.solr.cluster.placement.impl;
+
+import org.apache.solr.cluster.*;
+import org.apache.solr.cluster.placement.AttributeFetcher;
+import org.apache.solr.cluster.placement.AttributeValues;
+import org.apache.solr.common.util.Pair;
+
+import java.util.*;
+
+/**
+ * Builder classes to make tests using different cluster and node configurations easier to write and to read.
+ */
+public class Builders {
+
+    public static ClusterBuilder newClusterBuilder() {
+        return new ClusterBuilder();
+    }
+
+    public static CollectionBuilder newCollectionBuilder(String collectionName) {
+        return new CollectionBuilder(collectionName);
+    }
+
+    static class ClusterBuilder {
+        private LinkedList<NodeBuilder> nodeBuilders = new LinkedList<>();
+        private LinkedList<CollectionBuilder> collectionBuilders = new LinkedList<>();
+
+        ClusterBuilder initializeNodes(int countNodes) {
+            nodeBuilders = new LinkedList<>();
+            for (int n = 0; n < countNodes; n++) {
+                nodeBuilders.add(new NodeBuilder().setNodeName("node" + n)); // Default name, can be changed
+            }
+            return this;
+        }
+
+        LinkedList<NodeBuilder> getNodeBuilders() {
+            return nodeBuilders;
+        }
+
+        ClusterBuilder addCollection(CollectionBuilder collectionBuilder) {
+            collectionBuilders.add(collectionBuilder);
+            return this;
+        }
+
+        Cluster build() {
+            // TODO if converting all tests to use builders change ClusterImpl ctor to use list of nodes
+            return new ClusterAbstractionsForTest.ClusterImpl(new HashSet<>(buildLiveNodes()), buildClusterCollections());
+        }
+
+        List<Node> buildLiveNodes() {
+            List<Node> liveNodes = new LinkedList<>();
+            for (NodeBuilder nodeBuilder : nodeBuilders) {
+                liveNodes.add(nodeBuilder.build());
+            }
+
+            return liveNodes;
+        }
+
+        Map<String, SolrCollection> buildClusterCollections() {
+            Map<String, SolrCollection> clusterCollections = new LinkedHashMap<>();
+            for (CollectionBuilder collectionBuilder : collectionBuilders) {
+                SolrCollection solrCollection = collectionBuilder.build();
+                clusterCollections.put(solrCollection.getName(), solrCollection);
+            }
+
+            return clusterCollections;
+        }
+
+        AttributeFetcher buildAttributeFetcher() {
+            Map<Node, Integer> nodeToCoreCount = new HashMap<>();
+            Map<Node, Long> nodeToFreeDisk = new HashMap<>();
+
+            // TODO And a few more missing and will be added...
+
+            // Slight redoing of work twice (building Node instances) but let's favor readability over tricks (I could think
+            // of many) to reuse the nodes computed in build() or build the AttributeFetcher at the same time.
+            for (NodeBuilder nodeBuilder : nodeBuilders) {
+                Node node = nodeBuilder.build();
+
+                if (nodeBuilder.getCoreCount() != null) {
+                    nodeToCoreCount.put(node, nodeBuilder.getCoreCount());
+                }
+                if (nodeBuilder.getFreeDiskGB() != null) {
+                    nodeToFreeDisk.put(node, nodeBuilder.getFreeDiskGB());
+                }
+            }
+
+            AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
+            return new AttributeFetcherForTest(attributeValues);
+        }
+    }
+
+    static class CollectionBuilder {
+        private final String collectionName;
+        private LinkedList<ShardBuilder> shardBuilders = new LinkedList<>();
+        private Map<String, String> customProperties = new HashMap<>();
+
+
+        private CollectionBuilder(String collectionName) {
+            this.collectionName = collectionName;
+        }
+
+        private CollectionBuilder addCustomProperty(String name, String value) {
+            customProperties.put(name, value);
+            return this;
+        }
+
+        /**
+         * Initializes shard and replica builders for the collection based on passed parameters. Replicas are assigned round
+         * robin to the nodes. The shard leader is the first NRT replica of each shard (or first TLOG is no NRT).
+         * Shard and replica configuration can be modified afterwards, the returned builder hierarchy is a convenient starting point.
+         */
+        CollectionBuilder initializeShardsReplicas(int countShards, int countNrtReplicas, int countTlogReplicas,
+                                                   int countPullReplicas, List<NodeBuilder> nodes) {
+            Iterator<NodeBuilder> nodeIterator = nodes.iterator();
+
+            shardBuilders = new LinkedList<>();
+
+            for (int s = 0; s < countShards; s++) {
+                String shardName = collectionName + "_s" + s;
+
+                LinkedList<ReplicaBuilder> replicas = new LinkedList<>();
+                ReplicaBuilder leader = null;
+
+                // Iterate on requested counts, NRT then TLOG then PULL. Leader chosen as first NRT (or first TLOG if no NRT)
+                List<Pair<Replica.ReplicaType, Integer>> replicaTypes = List.of(
+                        new Pair<>(Replica.ReplicaType.NRT, countNrtReplicas),
+                        new Pair<>(Replica.ReplicaType.TLOG, countTlogReplicas),
+                        new Pair<>(Replica.ReplicaType.PULL, countPullReplicas));
+
+                for (Pair<Replica.ReplicaType, Integer> tc : replicaTypes) {
+                    Replica.ReplicaType type = tc.first();
+                    int count = tc.second();
+                    String replicaPrefix = shardName + "_" + type.name() + "_";
+                    for (int r = 0; r < count; r++) {
+                        String replicaName = replicaPrefix + r;
+                        String coreName = replicaName + "_c";
+                        if (!nodeIterator.hasNext()) {
+                            nodeIterator = nodes.iterator();
+                        }
+                        // If the nodes set is empty, this call will fail
+                        final NodeBuilder node = nodeIterator.next();
+
+                        ReplicaBuilder replicaBuilder = new ReplicaBuilder();
+                        replicaBuilder.setReplicaName(replicaName).setCoreName(coreName).setReplicaType(type)
+                                .setReplicaState(Replica.ReplicaState.ACTIVE).setReplicaNode(node);
+                        replicas.add(replicaBuilder);
+
+                        if (leader == null && type != Replica.ReplicaType.PULL) {
+                            leader = replicaBuilder;
+                        }
+                    }
+                }
+
+                ShardBuilder shardBuilder = new ShardBuilder();
+                shardBuilder.setShardName(shardName).setReplicaBuilders(replicas).setLeader(leader);
+                shardBuilders.add(shardBuilder);
+            }
+
+            return this;
+        }
+
+        SolrCollection build() {
+            ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(collectionName, customProperties);
+
+            final LinkedHashMap<String, Shard> shards = new LinkedHashMap<>();
+
+            for (ShardBuilder shardBuilder : shardBuilders) {
+                Shard shard = shardBuilder.build(solrCollection);
+                shards.put(shard.getShardName(), shard);
+            }
+
+            solrCollection.setShards(shards);
+            return solrCollection;
+        }
+    }
+
+    static class ShardBuilder {
+        private String shardName;
+        private LinkedList<ReplicaBuilder> replicaBuilders = new LinkedList<>();
+        private ReplicaBuilder leaderReplicaBuilder;
+
+        ShardBuilder setShardName(String shardName) {
+            this.shardName = shardName;
+            return this;
+        }
+
+        ShardBuilder setReplicaBuilders(LinkedList<ReplicaBuilder> replicaBuilders) {
+            this.replicaBuilders = replicaBuilders;
+            return this;
+        }
+
+        ShardBuilder setLeader(ReplicaBuilder leaderReplicaBuilder) {
+            this.leaderReplicaBuilder = leaderReplicaBuilder;
+            return this;
+        }
+
+        Shard build(SolrCollection collection) {
+            ClusterAbstractionsForTest.ShardImpl shard = new ClusterAbstractionsForTest.ShardImpl(shardName, collection, Shard.ShardState.ACTIVE);
+
+            final LinkedHashMap<String, Replica> replicas = new LinkedHashMap<>();
+            Replica leader = null;
+
+            for (ReplicaBuilder replicaBuilder : replicaBuilders) {
+                Replica replica = replicaBuilder.build(shard);
+                replicas.put(replica.getReplicaName(), replica);
+
+                if (leaderReplicaBuilder == replicaBuilder) {
+                    leader = replica;
+                }
+            }
+
+            shard.setReplicas(replicas, leader);
+            return shard;
+        }
+    }
+
+    static class ReplicaBuilder {
+        private String replicaName;
+        private String coreName;
+        private Replica.ReplicaType replicaType;
+        private Replica.ReplicaState replicaState;
+        private NodeBuilder replicaNode;
+
+        ReplicaBuilder setReplicaName(String replicaName) {
+            this.replicaName = replicaName;
+            return this;
+        }
+
+        ReplicaBuilder setCoreName(String coreName) {
+            this.coreName = coreName;
+            return this;
+        }
+
+        ReplicaBuilder setReplicaType(Replica.ReplicaType replicaType) {
+            this.replicaType = replicaType;
+            return this;
+        }
+
+        ReplicaBuilder setReplicaState(Replica.ReplicaState replicaState) {
+            this.replicaState = replicaState;
+            return this;
+        }
+
+        ReplicaBuilder setReplicaNode(NodeBuilder replicaNode) {
+            this.replicaNode = replicaNode;
+            return this;
+        }
+
+        Replica build(Shard shard) {
+            return new ClusterAbstractionsForTest.ReplicaImpl(replicaName, coreName, shard, replicaType, replicaState, replicaNode.build());
+        }
+    }
+
+    static class NodeBuilder {
+        private String nodeName = null;
+        private Integer coreCount = null;
+        private Long freeDiskGB = null;
+
+        NodeBuilder setNodeName(String nodeName) {
+            this.nodeName = nodeName;
+            return this;
+        }
+
+        NodeBuilder setCoreCount(Integer coreCount) {
+            this.coreCount = coreCount;
+            return this;
+        }
+
+        NodeBuilder setFreeDiskGB(Long freeDiskGB) {
+            this.freeDiskGB = freeDiskGB;
+            return this;
+        }
+
+        Integer getCoreCount() {
+            return coreCount;
+        }
+
+        Long getFreeDiskGB() {
+            return freeDiskGB;
+        }
+
+        Node build() {
+            // It is ok to build a new instance each time, that instance does the right thing with equals() and hashCode()
+            return new ClusterAbstractionsForTest.NodeImpl(nodeName);
+        }
+    }
+}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
index 188e3c3..69d3238 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
@@ -33,7 +33,7 @@ class ClusterAbstractionsForTest {
         private final Set<Node> liveNodes;
         private final Map<String, SolrCollection> collections;
 
-        ClusterImpl(Set<Node> liveNodes, Map<String, SolrCollection> collections) throws IOException {
+        ClusterImpl(Set<Node> liveNodes, Map<String, SolrCollection> collections) {
             this.liveNodes = liveNodes;
             this.collections = collections;
         }
@@ -88,7 +88,7 @@ class ClusterAbstractionsForTest {
         /**
          * This class ends up as a key in Maps in {@link org.apache.solr.cluster.placement.AttributeValues}.
          * It is important to implement this method comparing node names given that new instances of {@link Node} are created
-         * with names equal to existing instances (See {@link ReplicaImpl} constructor).
+         * with names equal to existing instances (See {@link Builders.NodeBuilder#build()}).
          */
         public boolean equals(Object obj) {
             if (obj == null) { return false; }

[lucene] 01/33: SOLR-15004: Unit test for SimpleClusterAbstractions.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 63607f47d919831e76406d644f3f3801720565e0
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue Nov 17 18:57:53 2020 +0100

    SOLR-15004: Unit test for SimpleClusterAbstractions.
---
 .../impl/SimpleClusterAbstractionsTest.java        | 88 ++++++++++++++++++++++
 1 file changed, 88 insertions(+)

diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsTest.java
new file mode 100644
index 0000000..eab2785
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.cluster.Cluster;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.Shard;
+import org.apache.solr.cluster.SolrCollection;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Slice;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Locale;
+import java.util.Set;
+
+/**
+ *
+ */
+public class SimpleClusterAbstractionsTest extends SolrCloudTestCase {
+
+  private static final String COLLECTION = SimpleClusterAbstractionsTest.class.getName() + "_collection";
+
+  private static SolrCloudManager cloudManager;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(3)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
+    CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 2)
+        .process(cluster.getSolrClient());
+  }
+
+  @Test
+  public void testBasic() throws Exception {
+    ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
+    Cluster cluster = new SimpleClusterAbstractionsImpl.ClusterImpl(cloudManager);
+    assertNotNull(cluster);
+    Set<Node> nodes = cluster.getLiveNodes();
+    nodes.forEach(n -> assertTrue("missing node " + n, clusterState.liveNodesContain(n.getName())));
+
+    DocCollection docCollection = clusterState.getCollection(COLLECTION);
+    SolrCollection collection = cluster.getCollection(COLLECTION);
+    // XXX gah ... can't assert anything about collection properties !!!??
+    // things like router or other collection props, like eg. special placement policy
+
+    assertNotNull(collection);
+    for (String shardName : docCollection.getSlicesMap().keySet()) {
+      Slice slice = docCollection.getSlice(shardName);
+      Shard shard = collection.getShard(shardName);
+      // XXX can't assert shard range ... because it's not in the API! :(
+
+      assertNotNull("missing shard " + shardName, shard);
+      assertNotNull("no leader in shard " + shard, shard.getLeader());
+      Replica replica = shard.getLeader();
+      assertEquals(slice.getLeader().getName(), replica.getReplicaName());
+      slice.getReplicas().forEach(sreplica -> {
+        Replica r = shard.getReplica(sreplica.getName());
+        assertNotNull("missing replica " + sreplica.getName(), r);
+        assertEquals(r.getCoreName(), sreplica.getCoreName());
+        assertEquals(r.getNode().getName(), sreplica.getNodeName());
+        assertEquals(r.getState().toString().toLowerCase(Locale.ROOT), sreplica.getState().toString());
+        assertEquals(r.getType().toString(), sreplica.getType().toString());
+      });
+    }
+  }
+}

[lucene] 25/33: SOLR-15016: Use '.' prefix for predefined plugins. Prevent errors when using unquoted JSON with string with a leading dot :) Make sure to process only plugin configs with the predefined names.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 88083a44f8ff6c3896404059662cd6a13191a68e
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Wed Dec 2 14:09:53 2020 +0100

    SOLR-15016: Use '.' prefix for predefined plugins.
    Prevent errors when using unquoted JSON with string with a leading dot :)
    Make sure to process only plugin configs with the predefined names.
---
 .../solr/cluster/events/ClusterEventProducer.java  |  2 +-
 .../events/impl/ClusterEventProducerFactory.java   | 32 ++++++++++++++--------
 .../cluster/placement/PlacementPluginFactory.java  |  2 +-
 .../impl/PlacementPluginFactoryLoader.java         | 17 ++++++++++--
 .../impl/PlacementPluginIntegrationTest.java       | 30 +++++++++++++++-----
 .../client/solrj/request/beans/PluginMeta.java     |  6 ++++
 6 files changed, 66 insertions(+), 23 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java b/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
index d3b0ee7..aa36fd7 100644
--- a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
+++ b/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
@@ -26,7 +26,7 @@ import java.io.Closeable;
 public interface ClusterEventProducer extends ClusterSingleton, Closeable {
 
   /** Unique name for the registration of a plugin-based implementation. */
-  String PLUGIN_NAME = "cluster-event-producer";
+  String PLUGIN_NAME = ".cluster-event-producer";
 
   @Override
   default String getName() {
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 85f1410..609f65c 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
@@ -128,13 +128,17 @@ public class ClusterEventProducerFactory extends ClusterEventProducerBase {
           ClusterEventListener listener = (ClusterEventListener) instance;
           clusterEventProducer.registerListener(listener);
         } else if (instance instanceof ClusterEventProducer) {
-          // replace the existing impl
-          if (cc.getClusterEventProducer() instanceof DelegatingClusterEventProducer) {
-            ((DelegatingClusterEventProducer) cc.getClusterEventProducer())
-                .setDelegate((ClusterEventProducer) instance);
+          if (ClusterEventProducer.PLUGIN_NAME.equals(plugin.getInfo().name)) {
+            // replace the existing impl
+            if (cc.getClusterEventProducer() instanceof DelegatingClusterEventProducer) {
+              ((DelegatingClusterEventProducer) cc.getClusterEventProducer())
+                  .setDelegate((ClusterEventProducer) instance);
+            } else {
+              log.warn("Can't configure plugin-based ClusterEventProducer while CoreContainer is still loading - " +
+                  " using existing implementation {}", cc.getClusterEventProducer().getClass().getName());
+            }
           } else {
-            log.warn("Can't configure plugin-based ClusterEventProducer while CoreContainer is still loading - " +
-                " using existing implementation {}", cc.getClusterEventProducer().getClass().getName());
+            log.warn("Ignoring ClusterEventProducer config with non-standard name: " + plugin.getInfo());
           }
         }
       }
@@ -149,13 +153,17 @@ public class ClusterEventProducerFactory extends ClusterEventProducerBase {
           ClusterEventListener listener = (ClusterEventListener) instance;
           clusterEventProducer.unregisterListener(listener);
         } else if (instance instanceof ClusterEventProducer) {
-          // replace the existing impl with NoOp
-          if (cc.getClusterEventProducer() instanceof DelegatingClusterEventProducer) {
-            ((DelegatingClusterEventProducer) cc.getClusterEventProducer())
-                .setDelegate(new NoOpProducer(cc));
+          if (ClusterEventProducer.PLUGIN_NAME.equals(plugin.getInfo().name)) {
+            // replace the existing impl with NoOp
+            if (cc.getClusterEventProducer() instanceof DelegatingClusterEventProducer) {
+              ((DelegatingClusterEventProducer) cc.getClusterEventProducer())
+                  .setDelegate(new NoOpProducer(cc));
+            } else {
+              log.warn("Can't configure plugin-based ClusterEventProducer while CoreContainer is still loading - " +
+                  " using existing implementation {}", cc.getClusterEventProducer().getClass().getName());
+            }
           } else {
-            log.warn("Can't configure plugin-based ClusterEventProducer while CoreContainer is still loading - " +
-                " using existing implementation {}", cc.getClusterEventProducer().getClass().getName());
+            log.warn("Ignoring ClusterEventProducer config with non-standard name: " + plugin.getInfo());
           }
         }
       }
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 abdd7b9..fc537ca 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
@@ -28,7 +28,7 @@ public interface PlacementPluginFactory {
   /**
    * The key in the plugins registry under which this plugin and its configuration are defined.
    */
-  String PLUGIN_NAME = "placement-plugin";
+  String PLUGIN_NAME = ".placement-plugin";
 
   /**
    * Returns an instance of the plugin that will be repeatedly (and concurrently) be called to compute placement. Multiple
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 8207279..46e6435 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
@@ -3,12 +3,17 @@ 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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.invoke.MethodHandles;
 
 /**
  * Utility class to load the configured {@link PlacementPluginFactory} plugin and
  * then keep it up to date as the plugin configuration changes.
  */
 public class PlacementPluginFactoryLoader {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   public static PlacementPluginFactory load(ContainerPluginsRegistry plugins) {
     final DelegatingPlacementPluginFactory pluginFactory = new DelegatingPlacementPluginFactory();
@@ -24,7 +29,11 @@ public class PlacementPluginFactoryLoader {
         }
         Object instance = plugin.getInstance();
         if (instance instanceof PlacementPluginFactory) {
-          pluginFactory.setDelegate((PlacementPluginFactory) instance);
+          if (PlacementPluginFactory.PLUGIN_NAME.equals(plugin.getInfo().name)) {
+            pluginFactory.setDelegate((PlacementPluginFactory) instance);
+          } else {
+            log.warn("Ignoring PlacementPluginFactory plugin with non-standard name: " + plugin.getInfo());
+          }
         }
       }
 
@@ -35,7 +44,11 @@ public class PlacementPluginFactoryLoader {
         }
         Object instance = plugin.getInstance();
         if (instance instanceof PlacementPluginFactory) {
-          pluginFactory.setDelegate(null);
+          if (PlacementPluginFactory.PLUGIN_NAME.equals(plugin.getInfo().name)) {
+            pluginFactory.setDelegate(null);
+          } else {
+            log.warn("Ignoring PlacementPluginFactory plugin with non-standard name: " + plugin.getInfo());
+          }
         }
       }
 
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 e3c9a27..6159358 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
@@ -84,7 +84,7 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
       req = new V2Request.Builder("/cluster/plugin")
           .forceV2(true)
           .POST()
-          .withPayload("{remove: " + PlacementPluginFactory.PLUGIN_NAME + "}")
+          .withPayload("{remove: '" + PlacementPluginFactory.PLUGIN_NAME + "'}")
           .build();
       req.process(cluster.getSolrClient());
     }
@@ -161,7 +161,7 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
         .build();
     req.process(cluster.getSolrClient());
 
-    version = waitForVersionChange(version, wrapper);
+    version = waitForVersionChange(version, wrapper, 10);
 
     factory = wrapper.getDelegate();
     assertTrue("wrong type " + factory.getClass().getName(), factory instanceof AffinityPlacementFactory);
@@ -178,27 +178,43 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
         .build();
     req.process(cluster.getSolrClient());
 
-    version = waitForVersionChange(version, wrapper);
+    version = waitForVersionChange(version, wrapper, 10);
     factory = wrapper.getDelegate();
     assertTrue("wrong type " + factory.getClass().getName(), factory instanceof AffinityPlacementFactory);
     config = ((AffinityPlacementFactory) factory).getConfig();
     assertEquals("minimalFreeDiskGB", 3, config.minimalFreeDiskGB);
     assertEquals("prioritizedFreeDiskGB", 4, config.prioritizedFreeDiskGB);
 
+    // add plugin of the right type but with the wrong name
+    plugin.name = "myPlugin";
+    req = new V2Request.Builder("/cluster/plugin")
+        .forceV2(true)
+        .POST()
+        .withPayload(singletonMap("add", plugin))
+        .build();
+    req.process(cluster.getSolrClient());
+    try {
+      int newVersion = waitForVersionChange(version, wrapper, 5);
+      if (newVersion != version) {
+        fail("factory configuration updated but plugin name was wrong: " + plugin);
+      }
+    } catch (TimeoutException te) {
+      // expected
+    }
     // remove plugin
     req = new V2Request.Builder("/cluster/plugin")
         .forceV2(true)
         .POST()
-        .withPayload("{remove: " + PlacementPluginFactory.PLUGIN_NAME + "}")
+        .withPayload("{remove: '" + PlacementPluginFactory.PLUGIN_NAME + "'}")
         .build();
     req.process(cluster.getSolrClient());
-    version = waitForVersionChange(version, wrapper);
+    version = waitForVersionChange(version, wrapper, 10);
     factory = wrapper.getDelegate();
     assertNull("no factory should be present", factory);
   }
 
-  private int waitForVersionChange(int currentVersion, PlacementPluginFactoryLoader.DelegatingPlacementPluginFactory wrapper) throws Exception {
-    TimeOut timeout = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+  private int waitForVersionChange(int currentVersion, PlacementPluginFactoryLoader.DelegatingPlacementPluginFactory wrapper, int timeoutSec) throws Exception {
+    TimeOut timeout = new TimeOut(timeoutSec, TimeUnit.SECONDS, TimeSource.NANO_TIME);
 
     while (!timeout.hasTimedOut()) {
       int newVersion = wrapper.getVersion();
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 5bee19f..3586ffa 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,6 +21,7 @@ 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
@@ -71,4 +72,9 @@ public class PluginMeta implements ReflectMapWriter {
   public int hashCode() {
     return Objects.hash(name, version, klass);
   }
+
+  @Override
+  public String toString() {
+    return Utils.toJSONString(this);
+  }
 }

[lucene] 15/33: Reformat to correct tab length (+ some other IntelliJ magic)

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 8b06bbbd35c4dd8049a9a59a221a8098047b2551
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Tue Nov 24 19:05:55 2020 +0100

    Reformat to correct tab length (+ some other IntelliJ magic)
---
 .../impl/AffinityPlacementFactoryTest.java         | 610 ++++++++++-----------
 .../placement/impl/AttributeFetcherForTest.java    | 130 ++---
 .../solr/cluster/placement/impl/Builders.java      | 516 ++++++++---------
 .../placement/impl/ClusterAbstractionsForTest.java | 600 ++++++++++----------
 .../cluster/placement/impl/PluginTestHelper.java   | 116 ++--
 5 files changed, 996 insertions(+), 976 deletions(-)

diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
index 5a6911e..3981b09 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
@@ -41,332 +41,332 @@ import java.util.stream.StreamSupport;
  * Unit test for {@link AffinityPlacementFactory}
  */
 public class AffinityPlacementFactoryTest extends Assert {
-    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-    private static PlacementPlugin plugin;
-
-    @BeforeClass
-    public static void setupPlugin() {
-        PlacementPluginConfig config = PlacementPluginConfigImpl.createConfigFromProperties(
-                Map.of("minimalFreeDiskGB", 10L, "deprioritizedFreeDiskGB", 50L));
-        plugin = new AffinityPlacementFactory().createPluginInstance(config);
-    }
-
-    @Test
-    public void testBasicPlacementNewCollection() throws Exception {
-        testBasicPlacementInternal(false);
-    }
-
-    @Test
-    public void testBasicPlacementExistingCollection() throws Exception {
-        testBasicPlacementInternal(true);
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static PlacementPlugin plugin;
+
+  @BeforeClass
+  public static void setupPlugin() {
+    PlacementPluginConfig config = PlacementPluginConfigImpl.createConfigFromProperties(
+        Map.of("minimalFreeDiskGB", 10L, "deprioritizedFreeDiskGB", 50L));
+    plugin = new AffinityPlacementFactory().createPluginInstance(config);
+  }
+
+  @Test
+  public void testBasicPlacementNewCollection() throws Exception {
+    testBasicPlacementInternal(false);
+  }
+
+  @Test
+  public void testBasicPlacementExistingCollection() throws Exception {
+    testBasicPlacementInternal(true);
+  }
+
+  @Test
+  public void testBasicPlacementNewCollection2() throws Exception {
+    testBasicInternal2(false);
+  }
+
+  @Test
+  public void testBasicPlacementExistingCollection2() throws Exception {
+    testBasicInternal2(true);
+  }
+
+  private void testBasicInternal2(boolean hasExistingCollection) throws Exception {
+    String collectionName = "testCollection";
+
+    Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(2);
+    LinkedList<Builders.NodeBuilder> nodeBuilders = clusterBuilder.getNodeBuilders();
+    nodeBuilders.get(0).setCoreCount(1).setFreeDiskGB(100L);
+    nodeBuilders.get(1).setCoreCount(10).setFreeDiskGB(100L);
+
+    Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
+
+    if (hasExistingCollection) {
+      // Existing collection has replicas for its shards and is visible in the cluster state
+      collectionBuilder.initializeShardsReplicas(1, 1, 0, 0, nodeBuilders);
+      clusterBuilder.addCollection(collectionBuilder);
+    } else {
+      // New collection to create has the shards defined but no replicas and is not present in cluster state
+      collectionBuilder.initializeShardsReplicas(1, 0, 0, 0, List.of());
     }
 
-    @Test
-    public void testBasicPlacementNewCollection2() throws Exception {
-        testBasicInternal2(false);
+    Cluster cluster = clusterBuilder.build();
+    AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
+
+    SolrCollection solrCollection = collectionBuilder.build();
+    List<Node> liveNodes = clusterBuilder.buildLiveNodes();
+
+    // Place a new replica for the (only) existing shard of the collection
+    PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
+        Set.of(solrCollection.shards().iterator().next().getShardName()), new HashSet<>(liveNodes),
+        1, 0, 0);
+
+    PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, new PlacementPlanFactoryImpl());
+
+    assertEquals(1, pp.getReplicaPlacements().size());
+    ReplicaPlacement rp = pp.getReplicaPlacements().iterator().next();
+    assertEquals(hasExistingCollection ? liveNodes.get(1) : liveNodes.get(0), rp.getNode());
+  }
+
+  /**
+   * When this test places a replica for a new collection, it should pick the node with less cores.<p>
+   * <p>
+   * When it places a replica for an existing collection, it should pick the node with more cores that doesn't already have a replica for the shard.
+   */
+  private void testBasicPlacementInternal(boolean hasExistingCollection) throws Exception {
+    String collectionName = "testCollection";
+
+    Node node1 = new ClusterAbstractionsForTest.NodeImpl("node1");
+    Node node2 = new ClusterAbstractionsForTest.NodeImpl("node2");
+    Set<Node> liveNodes = Set.of(node1, node2);
+
+    ClusterAbstractionsForTest.SolrCollectionImpl solrCollection;
+    // Make sure new collections are not visible in the cluster state and existing ones are
+    final Map<String, SolrCollection> clusterCollections;
+    if (hasExistingCollection) {
+      // An existing collection with a single replica on node 1. Note that new collections already exist by the time the plugin is called, but are empty
+      solrCollection = PluginTestHelper.createCollection(collectionName, Map.of(), 1, 1, 0, 0, Set.of(node1));
+      clusterCollections = Map.of(solrCollection.getName(), solrCollection);
+    } else {
+      // A new collection has the shards defined ok but no replicas
+      solrCollection = PluginTestHelper.createCollection(collectionName, Map.of(), 1, 0, 0, 0, Set.of());
+      clusterCollections = Map.of();
     }
 
-    @Test
-    public void testBasicPlacementExistingCollection2() throws Exception {
-        testBasicInternal2(true);
+    Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, clusterCollections);
+    // Place a new replica for the (only) existing shard of the collection
+    PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, Set.of(solrCollection.shards().iterator().next().getShardName()), liveNodes, 1, 0, 0);
+    // More cores on node2
+    Map<Node, Integer> nodeToCoreCount = Map.of(node1, 1, node2, 10);
+    // A lot of free disk on the two nodes
+    final Map<Node, Long> nodeToFreeDisk = Map.of(node1, 100L, node2, 100L);
+    AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
+    AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
+    PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+
+    PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+
+
+    assertEquals(1, pp.getReplicaPlacements().size());
+    ReplicaPlacement rp = pp.getReplicaPlacements().iterator().next();
+    assertEquals(hasExistingCollection ? node2 : node1, rp.getNode());
+  }
+
+  @Test
+  public void testAvailabilityZones() throws Exception {
+    String collectionName = "testCollection";
+    int NUM_NODES = 6;
+    Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(NUM_NODES);
+    for (int i = 0; i < NUM_NODES; i++) {
+      Builders.NodeBuilder nodeBuilder = clusterBuilder.getNodeBuilders().get(i);
+      nodeBuilder.setCoreCount(0);
+      nodeBuilder.setFreeDiskGB(100L);
+      if (i < NUM_NODES / 2) {
+        nodeBuilder.setSysprop(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP, "az1");
+      } else {
+        nodeBuilder.setSysprop(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP, "az2");
+      }
     }
 
-    private void testBasicInternal2(boolean hasExistingCollection) throws Exception {
-        String collectionName = "testCollection";
-
-        Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(2);
-        LinkedList<Builders.NodeBuilder> nodeBuilders = clusterBuilder.getNodeBuilders();
-        nodeBuilders.get(0).setCoreCount(1).setFreeDiskGB(100L);
-        nodeBuilders.get(1).setCoreCount(10).setFreeDiskGB(100L);
-
-        Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
-
-        if (hasExistingCollection) {
-            // Existing collection has replicas for its shards and is visible in the cluster state
-            collectionBuilder.initializeShardsReplicas(1, 1, 0, 0, nodeBuilders);
-            clusterBuilder.addCollection(collectionBuilder);
-        } else {
-            // New collection to create has the shards defined but no replicas and is not present in cluster state
-            collectionBuilder.initializeShardsReplicas(1, 0, 0, 0, List.of());
-        }
-
-        Cluster cluster = clusterBuilder.build();
-        AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
-
-        SolrCollection solrCollection = collectionBuilder.build();
-        List<Node> liveNodes = clusterBuilder.buildLiveNodes();
+    Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
+    collectionBuilder.initializeShardsReplicas(2, 0, 0, 0, clusterBuilder.getNodeBuilders());
+    clusterBuilder.addCollection(collectionBuilder);
 
-        // Place a new replica for the (only) existing shard of the collection
-        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
-                Set.of(solrCollection.shards().iterator().next().getShardName()), new HashSet<>(liveNodes),
-                1, 0, 0);
+    Cluster cluster = clusterBuilder.build();
 
-        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, new PlacementPlanFactoryImpl());
+    SolrCollection solrCollection = cluster.getCollection(collectionName);
 
-        assertEquals(1, pp.getReplicaPlacements().size());
-        ReplicaPlacement rp = pp.getReplicaPlacements().iterator().next();
-        assertEquals(hasExistingCollection ? liveNodes.get(1) : liveNodes.get(0), rp.getNode());
-    }
-
-    /**
-     * When this test places a replica for a new collection, it should pick the node with less cores.<p>
-     *
-     * When it places a replica for an existing collection, it should pick the node with more cores that doesn't already have a replica for the shard.
-     */
-    private void testBasicPlacementInternal(boolean hasExistingCollection) throws Exception {
-        String collectionName = "testCollection";
-
-        Node node1 = new ClusterAbstractionsForTest.NodeImpl("node1");
-        Node node2 = new ClusterAbstractionsForTest.NodeImpl("node2");
-        Set<Node> liveNodes = Set.of(node1, node2);
-
-        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection;
-        // Make sure new collections are not visible in the cluster state and existing ones are
-        final Map<String, SolrCollection> clusterCollections;
-        if (hasExistingCollection) {
-            // An existing collection with a single replica on node 1. Note that new collections already exist by the time the plugin is called, but are empty
-            solrCollection = PluginTestHelper.createCollection(collectionName, Map.of(), 1, 1, 0, 0, Set.of(node1));
-            clusterCollections = Map.of(solrCollection.getName(), solrCollection);
-        } else {
-            // A new collection has the shards defined ok but no replicas
-            solrCollection = PluginTestHelper.createCollection(collectionName, Map.of(), 1, 0, 0, 0, Set.of());
-            clusterCollections = Map.of();
-        }
-
-        Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, clusterCollections);
-        // Place a new replica for the (only) existing shard of the collection
-        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, Set.of(solrCollection.shards().iterator().next().getShardName()), liveNodes, 1, 0, 0);
-        // More cores on node2
-        Map<Node, Integer> nodeToCoreCount = Map.of(node1, 1, node2, 10);
-        // A lot of free disk on the two nodes
-        final Map<Node, Long> nodeToFreeDisk = Map.of(node1, 100L, node2, 100L);
-        AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
-        AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
-        PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
-
-        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
-
-
-        assertEquals(1, pp.getReplicaPlacements().size());
-        ReplicaPlacement rp = pp.getReplicaPlacements().iterator().next();
-        assertEquals(hasExistingCollection ? node2 : node1, rp.getNode());
-    }
+    PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
+        StreamSupport.stream(solrCollection.shards().spliterator(), false)
+            .map(Shard::getShardName).collect(Collectors.toSet()),
+        cluster.getLiveNodes(), 2, 2, 2);
 
-    @Test
-    public void testAvailabilityZones() throws Exception {
-        String collectionName = "testCollection";
-        int NUM_NODES = 6;
-        Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(NUM_NODES);
-        for (int i = 0; i < NUM_NODES; i++) {
-            Builders.NodeBuilder nodeBuilder = clusterBuilder.getNodeBuilders().get(i);
-            nodeBuilder.setCoreCount(0);
-            nodeBuilder.setFreeDiskGB(100L);
-            if (i < NUM_NODES / 2) {
-                nodeBuilder.setSysprop(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP, "az1");
-            } else {
-                nodeBuilder.setSysprop(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP, "az2");
-            }
-        }
-
-        Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
-        collectionBuilder.initializeShardsReplicas(2, 0, 0, 0, clusterBuilder.getNodeBuilders());
-        clusterBuilder.addCollection(collectionBuilder);
-
-        Cluster cluster = clusterBuilder.build();
-
-        SolrCollection solrCollection = cluster.getCollection(collectionName);
-
-        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
-            StreamSupport.stream(solrCollection.shards().spliterator(), false)
-                 .map(Shard::getShardName).collect(Collectors.toSet()),
-            cluster.getLiveNodes(), 2, 2, 2);
-
-        PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
-        AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
-        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
-        // 2 shards, 6 replicas
-        assertEquals(12, pp.getReplicaPlacements().size());
+    PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+    AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
+    PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+    // 2 shards, 6 replicas
+    assertEquals(12, pp.getReplicaPlacements().size());
 //        List<ReplicaPlacement> placements = new ArrayList<>(pp.getReplicaPlacements());
 //        Collections.sort(placements, Comparator
 //            .comparing((ReplicaPlacement p) -> p.getNode().getName())
 //            .thenComparing((ReplicaPlacement p) -> p.getShardName())
 //            .thenComparing((ReplicaPlacement p) -> p.getReplicaType())
 //        );
-        // shard -> AZ -> replica count
-        Map<Replica.ReplicaType, Map<String, Map<String, AtomicInteger>>> replicas = new HashMap<>();
-        AttributeValues attributeValues = attributeFetcher.fetchAttributes();
-        for (ReplicaPlacement rp : pp.getReplicaPlacements()) {
-            Optional<String> azOptional = attributeValues.getSystemProperty(rp.getNode(), AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP);
-            if (!azOptional.isPresent()) {
-                fail("missing AZ sysprop for node " + rp.getNode());
-            }
-            String az = azOptional.get();
-            replicas.computeIfAbsent(rp.getReplicaType(), type -> new HashMap<>())
-                .computeIfAbsent(rp.getShardName(), shard -> new HashMap<>())
-                .computeIfAbsent(az, zone -> new AtomicInteger()).incrementAndGet();
-        }
-        replicas.forEach((type, perTypeReplicas) -> {
-            perTypeReplicas.forEach((shard, azCounts) -> {
-                assertEquals("number of AZs", 2, azCounts.size());
-                azCounts.forEach((az, count) -> {
-                    assertTrue("too few replicas shard=" + shard + ", type=" + type + ", az=" + az,
-                        count.get() >= 1);
-                });
-            });
-        });
+    // shard -> AZ -> replica count
+    Map<Replica.ReplicaType, Map<String, Map<String, AtomicInteger>>> replicas = new HashMap<>();
+    AttributeValues attributeValues = attributeFetcher.fetchAttributes();
+    for (ReplicaPlacement rp : pp.getReplicaPlacements()) {
+      Optional<String> azOptional = attributeValues.getSystemProperty(rp.getNode(), AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP);
+      if (!azOptional.isPresent()) {
+        fail("missing AZ sysprop for node " + rp.getNode());
+      }
+      String az = azOptional.get();
+      replicas.computeIfAbsent(rp.getReplicaType(), type -> new HashMap<>())
+          .computeIfAbsent(rp.getShardName(), shard -> new HashMap<>())
+          .computeIfAbsent(az, zone -> new AtomicInteger()).incrementAndGet();
     }
-
-    @Test
-    public void testReplicaType() throws Exception {
-        String collectionName = "testCollection";
-        int NUM_NODES = 6;
-        Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(NUM_NODES);
-        for (int i = 0; i < NUM_NODES; i++) {
-            Builders.NodeBuilder nodeBuilder = clusterBuilder.getNodeBuilders().get(i);
-            nodeBuilder.setCoreCount(0);
-            nodeBuilder.setFreeDiskGB(100L);
-            if (i < NUM_NODES / 2) {
-                nodeBuilder.setSysprop(AffinityPlacementFactory.REPLICA_TYPE_SYSPROP, "Nrt,Tlog");
-                nodeBuilder.setSysprop("group", "one");
-            } else {
-                nodeBuilder.setSysprop(AffinityPlacementFactory.REPLICA_TYPE_SYSPROP, "Pull, foobar");
-                nodeBuilder.setSysprop("group", "two");
-            }
-        }
-
-        Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
-        collectionBuilder.initializeShardsReplicas(2, 0, 0, 0, clusterBuilder.getNodeBuilders());
-        clusterBuilder.addCollection(collectionBuilder);
-
-        Cluster cluster = clusterBuilder.build();
-
-        SolrCollection solrCollection = cluster.getCollection(collectionName);
-
-        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
-            StreamSupport.stream(solrCollection.shards().spliterator(), false)
-                .map(Shard::getShardName).collect(Collectors.toSet()),
-            cluster.getLiveNodes(), 2, 2, 2);
-
-        PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
-        AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
-        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
-        // 2 shards, 6 replicas
-        assertEquals(12, pp.getReplicaPlacements().size());
-        // shard -> group -> replica count
-        Map<Replica.ReplicaType, Map<String, Map<String, AtomicInteger>>> replicas = new HashMap<>();
-        AttributeValues attributeValues = attributeFetcher.fetchAttributes();
-        for (ReplicaPlacement rp : pp.getReplicaPlacements()) {
-            Optional<String> groupOptional = attributeValues.getSystemProperty(rp.getNode(), "group");
-            if (!groupOptional.isPresent()) {
-                fail("missing group sysprop for node " + rp.getNode());
-            }
-            String group = groupOptional.get();
-            if (group.equals("one")) {
-                assertTrue("wrong replica type in group one",
-                    (rp.getReplicaType() == Replica.ReplicaType.NRT) || rp.getReplicaType() == Replica.ReplicaType.TLOG);
-            } else {
-                assertEquals("wrong replica type in group two", Replica.ReplicaType.PULL, rp.getReplicaType());
-            }
-            replicas.computeIfAbsent(rp.getReplicaType(), type -> new HashMap<>())
-                .computeIfAbsent(rp.getShardName(), shard -> new HashMap<>())
-                .computeIfAbsent(group, g -> new AtomicInteger()).incrementAndGet();
-        }
-        replicas.forEach((type, perTypeReplicas) -> {
-            perTypeReplicas.forEach((shard, groupCounts) -> {
-                assertEquals("number of groups", 1, groupCounts.size());
-                groupCounts.forEach((group, count) -> {
-                    assertTrue("too few replicas shard=" + shard + ", type=" + type + ", group=" + group,
-                        count.get() >= 1);
-                });
-            });
+    replicas.forEach((type, perTypeReplicas) -> {
+      perTypeReplicas.forEach((shard, azCounts) -> {
+        assertEquals("number of AZs", 2, azCounts.size());
+        azCounts.forEach((az, count) -> {
+          assertTrue("too few replicas shard=" + shard + ", type=" + type + ", az=" + az,
+              count.get() >= 1);
         });
-
+      });
+    });
+  }
+
+  @Test
+  public void testReplicaType() throws Exception {
+    String collectionName = "testCollection";
+    int NUM_NODES = 6;
+    Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(NUM_NODES);
+    for (int i = 0; i < NUM_NODES; i++) {
+      Builders.NodeBuilder nodeBuilder = clusterBuilder.getNodeBuilders().get(i);
+      nodeBuilder.setCoreCount(0);
+      nodeBuilder.setFreeDiskGB(100L);
+      if (i < NUM_NODES / 2) {
+        nodeBuilder.setSysprop(AffinityPlacementFactory.REPLICA_TYPE_SYSPROP, "Nrt,Tlog");
+        nodeBuilder.setSysprop("group", "one");
+      } else {
+        nodeBuilder.setSysprop(AffinityPlacementFactory.REPLICA_TYPE_SYSPROP, "Pull, foobar");
+        nodeBuilder.setSysprop("group", "two");
+      }
     }
 
-    @Test
-    //@Ignore
-    public void testScalability() throws Exception {
-        log.info("==== numNodes ====");
-        runTestScalability(1000, 100, 40, 40, 20);
-        runTestScalability(2000, 100, 40, 40, 20);
-        runTestScalability(5000, 100, 40, 40, 20);
-        runTestScalability(10000, 100, 40, 40, 20);
-        runTestScalability(20000, 100, 40, 40, 20);
-        log.info("==== numShards ====");
-        runTestScalability(5000, 100, 40, 40, 20);
-        runTestScalability(5000, 200, 40, 40, 20);
-        runTestScalability(5000, 500, 40, 40, 20);
-        runTestScalability(5000, 1000, 40, 40, 20);
-        runTestScalability(5000, 2000, 40, 40, 20);
-        log.info("==== numReplicas ====");
-        runTestScalability(5000, 100, 100, 0, 0);
-        runTestScalability(5000, 100, 200, 0, 0);
-        runTestScalability(5000, 100, 500, 0, 0);
-        runTestScalability(5000, 100, 1000, 0, 0);
-        runTestScalability(5000, 100, 2000, 0, 0);
+    Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
+    collectionBuilder.initializeShardsReplicas(2, 0, 0, 0, clusterBuilder.getNodeBuilders());
+    clusterBuilder.addCollection(collectionBuilder);
+
+    Cluster cluster = clusterBuilder.build();
+
+    SolrCollection solrCollection = cluster.getCollection(collectionName);
+
+    PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
+        StreamSupport.stream(solrCollection.shards().spliterator(), false)
+            .map(Shard::getShardName).collect(Collectors.toSet()),
+        cluster.getLiveNodes(), 2, 2, 2);
+
+    PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+    AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
+    PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+    // 2 shards, 6 replicas
+    assertEquals(12, pp.getReplicaPlacements().size());
+    // shard -> group -> replica count
+    Map<Replica.ReplicaType, Map<String, Map<String, AtomicInteger>>> replicas = new HashMap<>();
+    AttributeValues attributeValues = attributeFetcher.fetchAttributes();
+    for (ReplicaPlacement rp : pp.getReplicaPlacements()) {
+      Optional<String> groupOptional = attributeValues.getSystemProperty(rp.getNode(), "group");
+      if (!groupOptional.isPresent()) {
+        fail("missing group sysprop for node " + rp.getNode());
+      }
+      String group = groupOptional.get();
+      if (group.equals("one")) {
+        assertTrue("wrong replica type in group one",
+            (rp.getReplicaType() == Replica.ReplicaType.NRT) || rp.getReplicaType() == Replica.ReplicaType.TLOG);
+      } else {
+        assertEquals("wrong replica type in group two", Replica.ReplicaType.PULL, rp.getReplicaType());
+      }
+      replicas.computeIfAbsent(rp.getReplicaType(), type -> new HashMap<>())
+          .computeIfAbsent(rp.getShardName(), shard -> new HashMap<>())
+          .computeIfAbsent(group, g -> new AtomicInteger()).incrementAndGet();
     }
-
-    private void runTestScalability(int numNodes, int numShards,
-                                    int nrtReplicas, int tlogReplicas,
-                                    int pullReplicas) throws Exception {
-
-        int REPLICAS_PER_SHARD = nrtReplicas + tlogReplicas + pullReplicas;
-        int TOTAL_REPLICAS = numShards * REPLICAS_PER_SHARD;
-
-        String collectionName = "testCollection";
-
-        final Set<Node> liveNodes = new HashSet<>();
-        final Map<Node, Long> nodeToFreeDisk = new HashMap<>();
-        final Map<Node, Integer> nodeToCoreCount = new HashMap<>();
-        for (int i = 0; i < numNodes; i++) {
-            Node node = new ClusterAbstractionsForTest.NodeImpl("node_" + i);
-            liveNodes.add(node);
-            nodeToFreeDisk.put(node, Long.valueOf(numNodes));
-            nodeToCoreCount.put(node, 0);
-        }
-        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection =
-            PluginTestHelper.createCollection(collectionName, Map.of(), numShards, 0, 0, 0, Set.of());
-
-        Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, Map.of());
-        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
-            // XXX awkward!
-            // StreamSupport.stream(solrCollection.shards().spliterator(), false)
-            //     .map(Shard::getShardName).collect(Collectors.toSet()),
-            solrCollection.getShardNames(),
-            liveNodes, nrtReplicas, tlogReplicas, pullReplicas);
-
-        AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
-        AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
-        PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
-
-        long start = System.nanoTime();
-        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
-        long end = System.nanoTime();
-        log.info("ComputePlacement: {} nodes, {} shards, {} total replicas, elapsed time {} ms.", numNodes, numShards, TOTAL_REPLICAS, TimeUnit.NANOSECONDS.toMillis(end - start)); //nowarn
-        assertEquals("incorrect number of calculated placements", TOTAL_REPLICAS,
-            pp.getReplicaPlacements().size());
-        // check that replicas are correctly placed
-        Map<Node, AtomicInteger> replicasPerNode = new HashMap<>();
-        Map<Node, Set<String>> shardsPerNode = new HashMap<>();
-        Map<String, AtomicInteger> replicasPerShard = new HashMap<>();
-        Map<Replica.ReplicaType, AtomicInteger> replicasByType = new HashMap<>();
-        for (ReplicaPlacement placement : pp.getReplicaPlacements()) {
-            replicasPerNode.computeIfAbsent(placement.getNode(), n -> new AtomicInteger()).incrementAndGet();
-            shardsPerNode.computeIfAbsent(placement.getNode(), n -> new HashSet<>()).add(placement.getShardName());
-            replicasByType.computeIfAbsent(placement.getReplicaType(), t -> new AtomicInteger()).incrementAndGet();
-            replicasPerShard.computeIfAbsent(placement.getShardName(), s -> new AtomicInteger()).incrementAndGet();
-        }
-        int perNode = TOTAL_REPLICAS > numNodes ? TOTAL_REPLICAS / numNodes : 1;
-        replicasPerNode.forEach((node, count) -> {
-            assertEquals(count.get(), perNode);
-        });
-        shardsPerNode.forEach((node, names) -> {
-            assertEquals(names.size(), perNode);
-        });
-
-        replicasPerShard.forEach((shard, count) -> {
-            assertEquals(count.get(), REPLICAS_PER_SHARD);
+    replicas.forEach((type, perTypeReplicas) -> {
+      perTypeReplicas.forEach((shard, groupCounts) -> {
+        assertEquals("number of groups", 1, groupCounts.size());
+        groupCounts.forEach((group, count) -> {
+          assertTrue("too few replicas shard=" + shard + ", type=" + type + ", group=" + group,
+              count.get() >= 1);
         });
+      });
+    });
+
+  }
+
+  @Test
+  //@Ignore
+  public void testScalability() throws Exception {
+    log.info("==== numNodes ====");
+    runTestScalability(1000, 100, 40, 40, 20);
+    runTestScalability(2000, 100, 40, 40, 20);
+    runTestScalability(5000, 100, 40, 40, 20);
+    runTestScalability(10000, 100, 40, 40, 20);
+    runTestScalability(20000, 100, 40, 40, 20);
+    log.info("==== numShards ====");
+    runTestScalability(5000, 100, 40, 40, 20);
+    runTestScalability(5000, 200, 40, 40, 20);
+    runTestScalability(5000, 500, 40, 40, 20);
+    runTestScalability(5000, 1000, 40, 40, 20);
+    runTestScalability(5000, 2000, 40, 40, 20);
+    log.info("==== numReplicas ====");
+    runTestScalability(5000, 100, 100, 0, 0);
+    runTestScalability(5000, 100, 200, 0, 0);
+    runTestScalability(5000, 100, 500, 0, 0);
+    runTestScalability(5000, 100, 1000, 0, 0);
+    runTestScalability(5000, 100, 2000, 0, 0);
+  }
+
+  private void runTestScalability(int numNodes, int numShards,
+                                  int nrtReplicas, int tlogReplicas,
+                                  int pullReplicas) throws Exception {
+
+    int REPLICAS_PER_SHARD = nrtReplicas + tlogReplicas + pullReplicas;
+    int TOTAL_REPLICAS = numShards * REPLICAS_PER_SHARD;
+
+    String collectionName = "testCollection";
+
+    final Set<Node> liveNodes = new HashSet<>();
+    final Map<Node, Long> nodeToFreeDisk = new HashMap<>();
+    final Map<Node, Integer> nodeToCoreCount = new HashMap<>();
+    for (int i = 0; i < numNodes; i++) {
+      Node node = new ClusterAbstractionsForTest.NodeImpl("node_" + i);
+      liveNodes.add(node);
+      nodeToFreeDisk.put(node, Long.valueOf(numNodes));
+      nodeToCoreCount.put(node, 0);
+    }
+    ClusterAbstractionsForTest.SolrCollectionImpl solrCollection =
+        PluginTestHelper.createCollection(collectionName, Map.of(), numShards, 0, 0, 0, Set.of());
+
+    Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, Map.of());
+    PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
+        // XXX awkward!
+        // StreamSupport.stream(solrCollection.shards().spliterator(), false)
+        //     .map(Shard::getShardName).collect(Collectors.toSet()),
+        solrCollection.getShardNames(),
+        liveNodes, nrtReplicas, tlogReplicas, pullReplicas);
+
+    AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
+    AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
+    PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+
+    long start = System.nanoTime();
+    PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+    long end = System.nanoTime();
+    log.info("ComputePlacement: {} nodes, {} shards, {} total replicas, elapsed time {} ms.", numNodes, numShards, TOTAL_REPLICAS, TimeUnit.NANOSECONDS.toMillis(end - start)); //nowarn
+    assertEquals("incorrect number of calculated placements", TOTAL_REPLICAS,
+        pp.getReplicaPlacements().size());
+    // check that replicas are correctly placed
+    Map<Node, AtomicInteger> replicasPerNode = new HashMap<>();
+    Map<Node, Set<String>> shardsPerNode = new HashMap<>();
+    Map<String, AtomicInteger> replicasPerShard = new HashMap<>();
+    Map<Replica.ReplicaType, AtomicInteger> replicasByType = new HashMap<>();
+    for (ReplicaPlacement placement : pp.getReplicaPlacements()) {
+      replicasPerNode.computeIfAbsent(placement.getNode(), n -> new AtomicInteger()).incrementAndGet();
+      shardsPerNode.computeIfAbsent(placement.getNode(), n -> new HashSet<>()).add(placement.getShardName());
+      replicasByType.computeIfAbsent(placement.getReplicaType(), t -> new AtomicInteger()).incrementAndGet();
+      replicasPerShard.computeIfAbsent(placement.getShardName(), s -> new AtomicInteger()).incrementAndGet();
     }
+    int perNode = TOTAL_REPLICAS > numNodes ? TOTAL_REPLICAS / numNodes : 1;
+    replicasPerNode.forEach((node, count) -> {
+      assertEquals(count.get(), perNode);
+    });
+    shardsPerNode.forEach((node, names) -> {
+      assertEquals(names.size(), perNode);
+    });
+
+    replicasPerShard.forEach((shard, count) -> {
+      assertEquals(count.get(), REPLICAS_PER_SHARD);
+    });
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java
index 58005f7..f053d1b 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java
@@ -25,69 +25,69 @@ import java.util.Set;
 
 public class AttributeFetcherForTest implements AttributeFetcher {
 
-    private final AttributeValues attributeValues;
-
-    AttributeFetcherForTest(AttributeValues attributeValues) {
-        this.attributeValues = attributeValues;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeCoreCount() {
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeDiskType() {
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeFreeDisk() {
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeTotalDisk() {
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeHeapUsage() {
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeSystemLoadAverage() {
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeSystemProperty(String name) {
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeEnvironmentVariable(String name) {
-        throw new UnsupportedOperationException("Not yet implemented...");
-    }
-
-    @Override
-    public AttributeFetcher requestNodeMetric(String metricName, NodeMetricRegistry registry) {
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher fetchFrom(Set<Node> nodes) {
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestMetric(String scope, String metricName) {
-        throw new UnsupportedOperationException("Not yet implemented...");
-    }
-
-    @Override
-    public AttributeValues fetchAttributes() {
-        return attributeValues;
-    }
+  private final AttributeValues attributeValues;
+
+  AttributeFetcherForTest(AttributeValues attributeValues) {
+    this.attributeValues = attributeValues;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeCoreCount() {
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeDiskType() {
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeFreeDisk() {
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeTotalDisk() {
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeHeapUsage() {
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeSystemLoadAverage() {
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeSystemProperty(String name) {
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeEnvironmentVariable(String name) {
+    throw new UnsupportedOperationException("Not yet implemented...");
+  }
+
+  @Override
+  public AttributeFetcher requestNodeMetric(String metricName, NodeMetricRegistry registry) {
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher fetchFrom(Set<Node> nodes) {
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestMetric(String scope, String metricName) {
+    throw new UnsupportedOperationException("Not yet implemented...");
+  }
+
+  @Override
+  public AttributeValues fetchAttributes() {
+    return attributeValues;
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
index bbbbc0e..1d964b2 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
@@ -12,318 +12,318 @@ import java.util.*;
  */
 public class Builders {
 
-    public static ClusterBuilder newClusterBuilder() {
-        return new ClusterBuilder();
+  public static ClusterBuilder newClusterBuilder() {
+    return new ClusterBuilder();
+  }
+
+  public static CollectionBuilder newCollectionBuilder(String collectionName) {
+    return new CollectionBuilder(collectionName);
+  }
+
+  static class ClusterBuilder {
+    private LinkedList<NodeBuilder> nodeBuilders = new LinkedList<>();
+    private LinkedList<CollectionBuilder> collectionBuilders = new LinkedList<>();
+
+    ClusterBuilder initializeNodes(int countNodes) {
+      nodeBuilders = new LinkedList<>();
+      for (int n = 0; n < countNodes; n++) {
+        nodeBuilders.add(new NodeBuilder().setNodeName("node" + n)); // Default name, can be changed
+      }
+      return this;
     }
 
-    public static CollectionBuilder newCollectionBuilder(String collectionName) {
-        return new CollectionBuilder(collectionName);
+    LinkedList<NodeBuilder> getNodeBuilders() {
+      return nodeBuilders;
     }
 
-    static class ClusterBuilder {
-        private LinkedList<NodeBuilder> nodeBuilders = new LinkedList<>();
-        private LinkedList<CollectionBuilder> collectionBuilders = new LinkedList<>();
-
-        ClusterBuilder initializeNodes(int countNodes) {
-            nodeBuilders = new LinkedList<>();
-            for (int n = 0; n < countNodes; n++) {
-                nodeBuilders.add(new NodeBuilder().setNodeName("node" + n)); // Default name, can be changed
-            }
-            return this;
-        }
+    ClusterBuilder addCollection(CollectionBuilder collectionBuilder) {
+      collectionBuilders.add(collectionBuilder);
+      return this;
+    }
 
-        LinkedList<NodeBuilder> getNodeBuilders() {
-            return nodeBuilders;
-        }
+    Cluster build() {
+      // TODO if converting all tests to use builders change ClusterImpl ctor to use list of nodes
+      return new ClusterAbstractionsForTest.ClusterImpl(new HashSet<>(buildLiveNodes()), buildClusterCollections());
+    }
 
-        ClusterBuilder addCollection(CollectionBuilder collectionBuilder) {
-            collectionBuilders.add(collectionBuilder);
-            return this;
-        }
+    List<Node> buildLiveNodes() {
+      List<Node> liveNodes = new LinkedList<>();
+      for (NodeBuilder nodeBuilder : nodeBuilders) {
+        liveNodes.add(nodeBuilder.build());
+      }
 
-        Cluster build() {
-            // TODO if converting all tests to use builders change ClusterImpl ctor to use list of nodes
-            return new ClusterAbstractionsForTest.ClusterImpl(new HashSet<>(buildLiveNodes()), buildClusterCollections());
-        }
+      return liveNodes;
+    }
 
-        List<Node> buildLiveNodes() {
-            List<Node> liveNodes = new LinkedList<>();
-            for (NodeBuilder nodeBuilder : nodeBuilders) {
-                liveNodes.add(nodeBuilder.build());
-            }
+    Map<String, SolrCollection> buildClusterCollections() {
+      Map<String, SolrCollection> clusterCollections = new LinkedHashMap<>();
+      for (CollectionBuilder collectionBuilder : collectionBuilders) {
+        SolrCollection solrCollection = collectionBuilder.build();
+        clusterCollections.put(solrCollection.getName(), solrCollection);
+      }
 
-            return liveNodes;
-        }
+      return clusterCollections;
+    }
 
-        Map<String, SolrCollection> buildClusterCollections() {
-            Map<String, SolrCollection> clusterCollections = new LinkedHashMap<>();
-            for (CollectionBuilder collectionBuilder : collectionBuilders) {
-                SolrCollection solrCollection = collectionBuilder.build();
-                clusterCollections.put(solrCollection.getName(), solrCollection);
-            }
+    AttributeFetcher buildAttributeFetcher() {
+      Map<Node, Integer> nodeToCoreCount = new HashMap<>();
+      Map<Node, Long> nodeToFreeDisk = new HashMap<>();
+      Map<String, Map<Node, String>> sysprops = new HashMap<>();
+      Map<String, Map<Node, Double>> metrics = new HashMap<>();
 
-            return clusterCollections;
-        }
+      // TODO And a few more missing and will be added...
 
-        AttributeFetcher buildAttributeFetcher() {
-            Map<Node, Integer> nodeToCoreCount = new HashMap<>();
-            Map<Node, Long> nodeToFreeDisk = new HashMap<>();
-            Map<String, Map<Node, String>> sysprops = new HashMap<>();
-            Map<String, Map<Node, Double>> metrics = new HashMap<>();
-
-            // TODO And a few more missing and will be added...
-
-            // Slight redoing of work twice (building Node instances) but let's favor readability over tricks (I could think
-            // of many) to reuse the nodes computed in build() or build the AttributeFetcher at the same time.
-            for (NodeBuilder nodeBuilder : nodeBuilders) {
-                Node node = nodeBuilder.build();
-
-                if (nodeBuilder.getCoreCount() != null) {
-                    nodeToCoreCount.put(node, nodeBuilder.getCoreCount());
-                }
-                if (nodeBuilder.getFreeDiskGB() != null) {
-                    nodeToFreeDisk.put(node, nodeBuilder.getFreeDiskGB());
-                }
-                if (nodeBuilder.getSysprops() != null) {
-                    nodeBuilder.getSysprops().forEach((name, value) -> {
-                        sysprops.computeIfAbsent(name, n -> new HashMap<>())
-                            .put(node, value);
-                    });
-                }
-                if (nodeBuilder.getMetrics() != null) {
-                    nodeBuilder.getMetrics().forEach((name, value) -> {
-                        metrics.computeIfAbsent(name, n -> new HashMap<>())
-                            .put(node, value);
-                    });
-                }
-            }
+      // Slight redoing of work twice (building Node instances) but let's favor readability over tricks (I could think
+      // of many) to reuse the nodes computed in build() or build the AttributeFetcher at the same time.
+      for (NodeBuilder nodeBuilder : nodeBuilders) {
+        Node node = nodeBuilder.build();
 
-            AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), sysprops, metrics);
-            return new AttributeFetcherForTest(attributeValues);
+        if (nodeBuilder.getCoreCount() != null) {
+          nodeToCoreCount.put(node, nodeBuilder.getCoreCount());
+        }
+        if (nodeBuilder.getFreeDiskGB() != null) {
+          nodeToFreeDisk.put(node, nodeBuilder.getFreeDiskGB());
+        }
+        if (nodeBuilder.getSysprops() != null) {
+          nodeBuilder.getSysprops().forEach((name, value) -> {
+            sysprops.computeIfAbsent(name, n -> new HashMap<>())
+                .put(node, value);
+          });
+        }
+        if (nodeBuilder.getMetrics() != null) {
+          nodeBuilder.getMetrics().forEach((name, value) -> {
+            metrics.computeIfAbsent(name, n -> new HashMap<>())
+                .put(node, value);
+          });
         }
+      }
+
+      AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), sysprops, metrics);
+      return new AttributeFetcherForTest(attributeValues);
     }
+  }
 
-    static class CollectionBuilder {
-        private final String collectionName;
-        private LinkedList<ShardBuilder> shardBuilders = new LinkedList<>();
-        private Map<String, String> customProperties = new HashMap<>();
+  static class CollectionBuilder {
+    private final String collectionName;
+    private LinkedList<ShardBuilder> shardBuilders = new LinkedList<>();
+    private Map<String, String> customProperties = new HashMap<>();
 
 
-        private CollectionBuilder(String collectionName) {
-            this.collectionName = collectionName;
-        }
+    private CollectionBuilder(String collectionName) {
+      this.collectionName = collectionName;
+    }
 
-        private CollectionBuilder addCustomProperty(String name, String value) {
-            customProperties.put(name, value);
-            return this;
-        }
+    private CollectionBuilder addCustomProperty(String name, String value) {
+      customProperties.put(name, value);
+      return this;
+    }
 
-        /**
-         * Initializes shard and replica builders for the collection based on passed parameters. Replicas are assigned round
-         * robin to the nodes. The shard leader is the first NRT replica of each shard (or first TLOG is no NRT).
-         * Shard and replica configuration can be modified afterwards, the returned builder hierarchy is a convenient starting point.
-         */
-        CollectionBuilder initializeShardsReplicas(int countShards, int countNrtReplicas, int countTlogReplicas,
-                                                   int countPullReplicas, List<NodeBuilder> nodes) {
-            Iterator<NodeBuilder> nodeIterator = nodes.iterator();
-
-            shardBuilders = new LinkedList<>();
-
-            for (int s = 0; s < countShards; s++) {
-                String shardName = "shard" + (s + 1);
-
-                LinkedList<ReplicaBuilder> replicas = new LinkedList<>();
-                ReplicaBuilder leader = null;
-
-                // Iterate on requested counts, NRT then TLOG then PULL. Leader chosen as first NRT (or first TLOG if no NRT)
-                List<Pair<Replica.ReplicaType, Integer>> replicaTypes = List.of(
-                        new Pair<>(Replica.ReplicaType.NRT, countNrtReplicas),
-                        new Pair<>(Replica.ReplicaType.TLOG, countTlogReplicas),
-                        new Pair<>(Replica.ReplicaType.PULL, countPullReplicas));
-
-                for (Pair<Replica.ReplicaType, Integer> tc : replicaTypes) {
-                    Replica.ReplicaType type = tc.first();
-                    int count = tc.second();
-                    String replicaPrefix = collectionName + "_" + shardName + "_replica_" + type.getSuffixChar();
-                    for (int r = 0; r < count; r++) {
-                        String replicaName = replicaPrefix + r;
-                        String coreName = replicaName + "_c";
-                        if (!nodeIterator.hasNext()) {
-                            nodeIterator = nodes.iterator();
-                        }
-                        // If the nodes set is empty, this call will fail
-                        final NodeBuilder node = nodeIterator.next();
-
-                        ReplicaBuilder replicaBuilder = new ReplicaBuilder();
-                        replicaBuilder.setReplicaName(replicaName).setCoreName(coreName).setReplicaType(type)
-                                .setReplicaState(Replica.ReplicaState.ACTIVE).setReplicaNode(node);
-                        replicas.add(replicaBuilder);
-
-                        if (leader == null && type != Replica.ReplicaType.PULL) {
-                            leader = replicaBuilder;
-                        }
-                    }
-                }
-
-                ShardBuilder shardBuilder = new ShardBuilder();
-                shardBuilder.setShardName(shardName).setReplicaBuilders(replicas).setLeader(leader);
-                shardBuilders.add(shardBuilder);
+    /**
+     * Initializes shard and replica builders for the collection based on passed parameters. Replicas are assigned round
+     * robin to the nodes. The shard leader is the first NRT replica of each shard (or first TLOG is no NRT).
+     * Shard and replica configuration can be modified afterwards, the returned builder hierarchy is a convenient starting point.
+     */
+    CollectionBuilder initializeShardsReplicas(int countShards, int countNrtReplicas, int countTlogReplicas,
+                                               int countPullReplicas, List<NodeBuilder> nodes) {
+      Iterator<NodeBuilder> nodeIterator = nodes.iterator();
+
+      shardBuilders = new LinkedList<>();
+
+      for (int s = 0; s < countShards; s++) {
+        String shardName = "shard" + (s + 1);
+
+        LinkedList<ReplicaBuilder> replicas = new LinkedList<>();
+        ReplicaBuilder leader = null;
+
+        // Iterate on requested counts, NRT then TLOG then PULL. Leader chosen as first NRT (or first TLOG if no NRT)
+        List<Pair<Replica.ReplicaType, Integer>> replicaTypes = List.of(
+            new Pair<>(Replica.ReplicaType.NRT, countNrtReplicas),
+            new Pair<>(Replica.ReplicaType.TLOG, countTlogReplicas),
+            new Pair<>(Replica.ReplicaType.PULL, countPullReplicas));
+
+        for (Pair<Replica.ReplicaType, Integer> tc : replicaTypes) {
+          Replica.ReplicaType type = tc.first();
+          int count = tc.second();
+          String replicaPrefix = collectionName + "_" + shardName + "_replica_" + type.getSuffixChar();
+          for (int r = 0; r < count; r++) {
+            String replicaName = replicaPrefix + r;
+            String coreName = replicaName + "_c";
+            if (!nodeIterator.hasNext()) {
+              nodeIterator = nodes.iterator();
             }
+            // If the nodes set is empty, this call will fail
+            final NodeBuilder node = nodeIterator.next();
 
-            return this;
-        }
-
-        SolrCollection build() {
-            ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(collectionName, customProperties);
+            ReplicaBuilder replicaBuilder = new ReplicaBuilder();
+            replicaBuilder.setReplicaName(replicaName).setCoreName(coreName).setReplicaType(type)
+                .setReplicaState(Replica.ReplicaState.ACTIVE).setReplicaNode(node);
+            replicas.add(replicaBuilder);
 
-            final LinkedHashMap<String, Shard> shards = new LinkedHashMap<>();
-
-            for (ShardBuilder shardBuilder : shardBuilders) {
-                Shard shard = shardBuilder.build(solrCollection);
-                shards.put(shard.getShardName(), shard);
+            if (leader == null && type != Replica.ReplicaType.PULL) {
+              leader = replicaBuilder;
             }
-
-            solrCollection.setShards(shards);
-            return solrCollection;
+          }
         }
-    }
 
-    static class ShardBuilder {
-        private String shardName;
-        private LinkedList<ReplicaBuilder> replicaBuilders = new LinkedList<>();
-        private ReplicaBuilder leaderReplicaBuilder;
+        ShardBuilder shardBuilder = new ShardBuilder();
+        shardBuilder.setShardName(shardName).setReplicaBuilders(replicas).setLeader(leader);
+        shardBuilders.add(shardBuilder);
+      }
 
-        ShardBuilder setShardName(String shardName) {
-            this.shardName = shardName;
-            return this;
-        }
+      return this;
+    }
 
-        ShardBuilder setReplicaBuilders(LinkedList<ReplicaBuilder> replicaBuilders) {
-            this.replicaBuilders = replicaBuilders;
-            return this;
-        }
+    SolrCollection build() {
+      ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(collectionName, customProperties);
 
-        ShardBuilder setLeader(ReplicaBuilder leaderReplicaBuilder) {
-            this.leaderReplicaBuilder = leaderReplicaBuilder;
-            return this;
-        }
+      final LinkedHashMap<String, Shard> shards = new LinkedHashMap<>();
 
-        Shard build(SolrCollection collection) {
-            ClusterAbstractionsForTest.ShardImpl shard = new ClusterAbstractionsForTest.ShardImpl(shardName, collection, Shard.ShardState.ACTIVE);
+      for (ShardBuilder shardBuilder : shardBuilders) {
+        Shard shard = shardBuilder.build(solrCollection);
+        shards.put(shard.getShardName(), shard);
+      }
 
-            final LinkedHashMap<String, Replica> replicas = new LinkedHashMap<>();
-            Replica leader = null;
+      solrCollection.setShards(shards);
+      return solrCollection;
+    }
+  }
 
-            for (ReplicaBuilder replicaBuilder : replicaBuilders) {
-                Replica replica = replicaBuilder.build(shard);
-                replicas.put(replica.getReplicaName(), replica);
+  static class ShardBuilder {
+    private String shardName;
+    private LinkedList<ReplicaBuilder> replicaBuilders = new LinkedList<>();
+    private ReplicaBuilder leaderReplicaBuilder;
 
-                if (leaderReplicaBuilder == replicaBuilder) {
-                    leader = replica;
-                }
-            }
+    ShardBuilder setShardName(String shardName) {
+      this.shardName = shardName;
+      return this;
+    }
 
-            shard.setReplicas(replicas, leader);
-            return shard;
-        }
+    ShardBuilder setReplicaBuilders(LinkedList<ReplicaBuilder> replicaBuilders) {
+      this.replicaBuilders = replicaBuilders;
+      return this;
     }
 
-    static class ReplicaBuilder {
-        private String replicaName;
-        private String coreName;
-        private Replica.ReplicaType replicaType;
-        private Replica.ReplicaState replicaState;
-        private NodeBuilder replicaNode;
+    ShardBuilder setLeader(ReplicaBuilder leaderReplicaBuilder) {
+      this.leaderReplicaBuilder = leaderReplicaBuilder;
+      return this;
+    }
 
-        ReplicaBuilder setReplicaName(String replicaName) {
-            this.replicaName = replicaName;
-            return this;
-        }
+    Shard build(SolrCollection collection) {
+      ClusterAbstractionsForTest.ShardImpl shard = new ClusterAbstractionsForTest.ShardImpl(shardName, collection, Shard.ShardState.ACTIVE);
 
-        ReplicaBuilder setCoreName(String coreName) {
-            this.coreName = coreName;
-            return this;
-        }
+      final LinkedHashMap<String, Replica> replicas = new LinkedHashMap<>();
+      Replica leader = null;
 
-        ReplicaBuilder setReplicaType(Replica.ReplicaType replicaType) {
-            this.replicaType = replicaType;
-            return this;
-        }
+      for (ReplicaBuilder replicaBuilder : replicaBuilders) {
+        Replica replica = replicaBuilder.build(shard);
+        replicas.put(replica.getReplicaName(), replica);
 
-        ReplicaBuilder setReplicaState(Replica.ReplicaState replicaState) {
-            this.replicaState = replicaState;
-            return this;
+        if (leaderReplicaBuilder == replicaBuilder) {
+          leader = replica;
         }
+      }
 
-        ReplicaBuilder setReplicaNode(NodeBuilder replicaNode) {
-            this.replicaNode = replicaNode;
-            return this;
-        }
+      shard.setReplicas(replicas, leader);
+      return shard;
+    }
+  }
+
+  static class ReplicaBuilder {
+    private String replicaName;
+    private String coreName;
+    private Replica.ReplicaType replicaType;
+    private Replica.ReplicaState replicaState;
+    private NodeBuilder replicaNode;
+
+    ReplicaBuilder setReplicaName(String replicaName) {
+      this.replicaName = replicaName;
+      return this;
+    }
 
-        Replica build(Shard shard) {
-            return new ClusterAbstractionsForTest.ReplicaImpl(replicaName, coreName, shard, replicaType, replicaState, replicaNode.build());
-        }
+    ReplicaBuilder setCoreName(String coreName) {
+      this.coreName = coreName;
+      return this;
     }
 
-    static class NodeBuilder {
-        private String nodeName = null;
-        private Integer coreCount = null;
-        private Long freeDiskGB = null;
-        private Map<String, String> sysprops = null;
-        private Map<String, Double> metrics = null;
+    ReplicaBuilder setReplicaType(Replica.ReplicaType replicaType) {
+      this.replicaType = replicaType;
+      return this;
+    }
 
-        NodeBuilder setNodeName(String nodeName) {
-            this.nodeName = nodeName;
-            return this;
-        }
+    ReplicaBuilder setReplicaState(Replica.ReplicaState replicaState) {
+      this.replicaState = replicaState;
+      return this;
+    }
 
-        NodeBuilder setCoreCount(Integer coreCount) {
-            this.coreCount = coreCount;
-            return this;
-        }
+    ReplicaBuilder setReplicaNode(NodeBuilder replicaNode) {
+      this.replicaNode = replicaNode;
+      return this;
+    }
 
-        NodeBuilder setFreeDiskGB(Long freeDiskGB) {
-            this.freeDiskGB = freeDiskGB;
-            return this;
-        }
+    Replica build(Shard shard) {
+      return new ClusterAbstractionsForTest.ReplicaImpl(replicaName, coreName, shard, replicaType, replicaState, replicaNode.build());
+    }
+  }
+
+  static class NodeBuilder {
+    private String nodeName = null;
+    private Integer coreCount = null;
+    private Long freeDiskGB = null;
+    private Map<String, String> sysprops = null;
+    private Map<String, Double> metrics = null;
+
+    NodeBuilder setNodeName(String nodeName) {
+      this.nodeName = nodeName;
+      return this;
+    }
 
-        NodeBuilder setSysprop(String key, String value) {
-            if (sysprops == null) {
-                sysprops = new HashMap<>();
-            }
-            String name = AttributeFetcherImpl.getSystemPropertySnitchTag(key);
-            sysprops.put(name, value);
-            return this;
-        }
+    NodeBuilder setCoreCount(Integer coreCount) {
+      this.coreCount = coreCount;
+      return this;
+    }
 
-        NodeBuilder setMetric(AttributeFetcher.NodeMetricRegistry registry, String key, Double value) {
-            if (metrics == null) {
-                metrics = new HashMap<>();
-            }
-            String name = AttributeFetcherImpl.getMetricSnitchTag(key, registry);
-            metrics.put(name, value);
-            return this;
-        }
+    NodeBuilder setFreeDiskGB(Long freeDiskGB) {
+      this.freeDiskGB = freeDiskGB;
+      return this;
+    }
 
-        Integer getCoreCount() {
-            return coreCount;
-        }
+    NodeBuilder setSysprop(String key, String value) {
+      if (sysprops == null) {
+        sysprops = new HashMap<>();
+      }
+      String name = AttributeFetcherImpl.getSystemPropertySnitchTag(key);
+      sysprops.put(name, value);
+      return this;
+    }
 
-        Long getFreeDiskGB() {
-            return freeDiskGB;
-        }
+    NodeBuilder setMetric(AttributeFetcher.NodeMetricRegistry registry, String key, Double value) {
+      if (metrics == null) {
+        metrics = new HashMap<>();
+      }
+      String name = AttributeFetcherImpl.getMetricSnitchTag(key, registry);
+      metrics.put(name, value);
+      return this;
+    }
 
-        Map<String, String> getSysprops() {
-            return sysprops;
-        }
+    Integer getCoreCount() {
+      return coreCount;
+    }
 
-        Map<String, Double> getMetrics() {
-            return metrics;
-        }
+    Long getFreeDiskGB() {
+      return freeDiskGB;
+    }
 
-        Node build() {
-            // It is ok to build a new instance each time, that instance does the right thing with equals() and hashCode()
-            return new ClusterAbstractionsForTest.NodeImpl(nodeName);
-        }
+    Map<String, String> getSysprops() {
+      return sysprops;
+    }
+
+    Map<String, Double> getMetrics() {
+      return metrics;
+    }
+
+    Node build() {
+      // It is ok to build a new instance each time, that instance does the right thing with equals() and hashCode()
+      return new ClusterAbstractionsForTest.NodeImpl(nodeName);
     }
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
index da82240..bd14d0d 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
@@ -28,295 +28,315 @@ import java.util.stream.Collectors;
  */
 class ClusterAbstractionsForTest {
 
-    static class ClusterImpl implements Cluster {
-        private final Set<Node> liveNodes = new HashSet<>();
-        private final Map<String, SolrCollection> collections = new HashMap<>();
-
-        ClusterImpl(Set<Node> liveNodes, Map<String, SolrCollection> collections) {
-            this.liveNodes.addAll(liveNodes);
-            this.collections.putAll(collections);
-        }
-
-        @Override
-        public Set<Node> getLiveNodes() {
-            return liveNodes;
-        }
-
-        @Override
-        public SolrCollection getCollection(String collectionName) {
-            return collections.get(collectionName);
-        }
-
-        @Override
-        @Nonnull
-        public Iterator<SolrCollection> iterator() {
-            return collections.values().iterator();
-        }
-
-        @Override
-        public Iterable<SolrCollection> collections() {
-            return ClusterImpl.this::iterator;
-        }
-
-        // for unit tests
-
-        ClusterImpl addNode(Node node) {
-            liveNodes.add(node);
-            return this;
-        }
-
-        ClusterImpl removeNode(Node node) {
-            liveNodes.remove(node);
-            return this;
-        }
-
-        ClusterImpl putCollection(SolrCollection collection) {
-            collections.put(collection.getName(), collection);
-            return this;
-        }
-
-        ClusterImpl removeCollection(String name) {
-            collections.remove(name);
-            return this;
-        }
-
-        ClusterImpl removeAllCollections() {
-            collections.clear();
-            return this;
-        }
-    }
-
-
-    static class NodeImpl implements Node {
-        public final String nodeName;
-
-        /**
-         * Transforms a collection of node names into a set of {@link Node} instances.
-         */
-        static Set<Node> getNodes(Collection<String> nodeNames) {
-            return nodeNames.stream().map(NodeImpl::new).collect(Collectors.toSet());
-        }
-
-        NodeImpl(String nodeName) {
-            this.nodeName = nodeName;
-        }
-
-        @Override
-        public String getName() {
-            return nodeName;
-        }
-
-        @Override
-        public String toString() {
-            return getClass().getSimpleName() + "(" + getName() + ")";
-        }
-
-        /**
-         * This class ends up as a key in Maps in {@link org.apache.solr.cluster.placement.AttributeValues}.
-         * It is important to implement this method comparing node names given that new instances of {@link Node} are created
-         * with names equal to existing instances (See {@link Builders.NodeBuilder#build()}).
-         */
-        public boolean equals(Object obj) {
-            if (obj == null) { return false; }
-            if (obj == this) { return true; }
-            if (obj.getClass() != getClass()) { return false; }
-            NodeImpl other = (NodeImpl) obj;
-            return Objects.equals(this.nodeName, other.nodeName);
-        }
-
-        public int hashCode() {
-            return Objects.hashCode(nodeName);
-        }
-    }
-
-
-    static class SolrCollectionImpl implements SolrCollection {
-        private final String collectionName;
-        /** Map from {@link Shard#getShardName()} to {@link Shard} */
-        private Map<String, Shard> shards;
-        private final Map<String, String> customProperties;
-
-        SolrCollectionImpl(String collectionName, Map<String, String> customProperties) {
-            this.collectionName = collectionName;
-            this.customProperties = customProperties;
-        }
-
-        /**
-         * Setting the shards has to happen (in tests) after creating the collection because shards reference the collection
-         */
-        void setShards(Map<String, Shard> shards) {
-            this.shards = shards;
-        }
-
-        Set<String> getShardNames() {
-            return shards.keySet();
-        }
-
-        @Override
-        public String getName() {
-            return collectionName;
-        }
-
-        @Override
-        public Shard getShard(String name) {
-            return shards.get(name);
-        }
-
-        @Override
-        @Nonnull
-        public Iterator<Shard> iterator() {
-            return shards.values().iterator();
-        }
-
-        @Override
-        public Iterable<Shard> shards() {
-            return SolrCollectionImpl.this::iterator;
-        }
-
-        @Override
-        public String getCustomProperty(String customPropertyName) {
-            return customProperties.get(customPropertyName);
-        }
-    }
-
-
-    static class ShardImpl implements Shard {
-        private final String shardName;
-        private final SolrCollection collection;
-        private final ShardState shardState;
-        private Map<String, Replica> replicas;
-        private Replica leader;
-
-        ShardImpl(String shardName, SolrCollection collection, ShardState shardState) {
-            this.shardName = shardName;
-            this.collection = collection;
-            this.shardState = shardState;
-        }
-
-        /**
-         * Setting the replicas has to happen (in tests) after creating the shard because replicas reference the shard
-         */
-        void setReplicas(Map<String, Replica> replicas, Replica leader) {
-            this.replicas = replicas;
-            this.leader = leader;
-        }
-
-        @Override
-        public String getShardName() {
-            return shardName;
-        }
-
-        @Override
-        public SolrCollection getCollection() {
-            return collection;
-        }
-
-        @Override
-        public Replica getReplica(String name) {
-            return replicas.get(name);
-        }
-
-        @Override
-        @Nonnull
-        public Iterator<Replica> iterator() {
-            return replicas.values().iterator();
-        }
-
-        @Override
-        public Iterable<Replica> replicas() {
-            return ShardImpl.this::iterator;
-        }
-
-        @Override
-        public Replica getLeader() {
-            return leader;
-        }
-
-        @Override
-        public ShardState getState() {
-            return shardState;
-        }
-
-        public boolean equals(Object obj) {
-            if (obj == null) { return false; }
-            if (obj == this) { return true; }
-            if (obj.getClass() != getClass()) { return false; }
-            ShardImpl other = (ShardImpl) obj;
-            return Objects.equals(this.shardName, other.shardName)
-                    && Objects.equals(this.collection, other.collection)
-                    && Objects.equals(this.shardState, other.shardState)
-                    && Objects.equals(this.replicas, other.replicas)
-                    && Objects.equals(this.leader, other.leader);
-        }
-
-        public int hashCode() {
-            return Objects.hash(shardName, collection, shardState);
-        }
-    }
-
-
-    static class ReplicaImpl implements Replica {
-        private final String replicaName;
-        private final String coreName;
-        private final Shard shard;
-        private final ReplicaType replicaType;
-        private final ReplicaState replicaState;
-        private final Node node;
-
-        ReplicaImpl(String replicaName, String coreName, Shard shard, ReplicaType replicaType, ReplicaState replicaState, Node node) {
-            this.replicaName = replicaName;
-            this.coreName = coreName;
-            this.shard = shard;
-            this.replicaType = replicaType;
-            this.replicaState = replicaState;
-            this.node = node;
-        }
-
-        @Override
-        public Shard getShard() {
-            return shard;
-        }
-
-        @Override
-        public ReplicaType getType() {
-            return replicaType;
-        }
-
-        @Override
-        public ReplicaState getState() {
-            return replicaState;
-        }
-
-        @Override
-        public String getReplicaName() {
-            return replicaName;
-        }
-
-        @Override
-        public String getCoreName() {
-            return coreName;
-        }
-
-        @Override
-        public Node getNode() {
-            return node;
-        }
-
-        public boolean equals(Object obj) {
-            if (obj == null) { return false; }
-            if (obj == this) { return true; }
-            if (obj.getClass() != getClass()) { return false; }
-            ReplicaImpl other = (ReplicaImpl) obj;
-            return Objects.equals(this.replicaName, other.replicaName)
-                    && Objects.equals(this.coreName, other.coreName)
-                    && Objects.equals(this.shard, other.shard)
-                    && Objects.equals(this.replicaType, other.replicaType)
-                    && Objects.equals(this.replicaState, other.replicaState)
-                    && Objects.equals(this.node, other.node);
-        }
-
-        public int hashCode() {
-            return Objects.hash(replicaName, coreName, shard, replicaType, replicaState, node);
-        }
+  static class ClusterImpl implements Cluster {
+    private final Set<Node> liveNodes = new HashSet<>();
+    private final Map<String, SolrCollection> collections = new HashMap<>();
+
+    ClusterImpl(Set<Node> liveNodes, Map<String, SolrCollection> collections) {
+      this.liveNodes.addAll(liveNodes);
+      this.collections.putAll(collections);
+    }
+
+    @Override
+    public Set<Node> getLiveNodes() {
+      return liveNodes;
+    }
+
+    @Override
+    public SolrCollection getCollection(String collectionName) {
+      return collections.get(collectionName);
+    }
+
+    @Override
+    @Nonnull
+    public Iterator<SolrCollection> iterator() {
+      return collections.values().iterator();
+    }
+
+    @Override
+    public Iterable<SolrCollection> collections() {
+      return ClusterImpl.this::iterator;
+    }
+
+    // for unit tests
+
+    ClusterImpl addNode(Node node) {
+      liveNodes.add(node);
+      return this;
+    }
+
+    ClusterImpl removeNode(Node node) {
+      liveNodes.remove(node);
+      return this;
+    }
+
+    ClusterImpl putCollection(SolrCollection collection) {
+      collections.put(collection.getName(), collection);
+      return this;
+    }
+
+    ClusterImpl removeCollection(String name) {
+      collections.remove(name);
+      return this;
+    }
+
+    ClusterImpl removeAllCollections() {
+      collections.clear();
+      return this;
+    }
+  }
+
+
+  static class NodeImpl implements Node {
+    public final String nodeName;
+
+    /**
+     * Transforms a collection of node names into a set of {@link Node} instances.
+     */
+    static Set<Node> getNodes(Collection<String> nodeNames) {
+      return nodeNames.stream().map(NodeImpl::new).collect(Collectors.toSet());
+    }
+
+    NodeImpl(String nodeName) {
+      this.nodeName = nodeName;
+    }
+
+    @Override
+    public String getName() {
+      return nodeName;
+    }
+
+    @Override
+    public String toString() {
+      return getClass().getSimpleName() + "(" + getName() + ")";
+    }
+
+    /**
+     * This class ends up as a key in Maps in {@link org.apache.solr.cluster.placement.AttributeValues}.
+     * It is important to implement this method comparing node names given that new instances of {@link Node} are created
+     * with names equal to existing instances (See {@link Builders.NodeBuilder#build()}).
+     */
+    public boolean equals(Object obj) {
+      if (obj == null) {
+        return false;
+      }
+      if (obj == this) {
+        return true;
+      }
+      if (obj.getClass() != getClass()) {
+        return false;
+      }
+      NodeImpl other = (NodeImpl) obj;
+      return Objects.equals(this.nodeName, other.nodeName);
+    }
+
+    public int hashCode() {
+      return Objects.hashCode(nodeName);
+    }
+  }
+
+
+  static class SolrCollectionImpl implements SolrCollection {
+    private final String collectionName;
+    /**
+     * Map from {@link Shard#getShardName()} to {@link Shard}
+     */
+    private Map<String, Shard> shards;
+    private final Map<String, String> customProperties;
+
+    SolrCollectionImpl(String collectionName, Map<String, String> customProperties) {
+      this.collectionName = collectionName;
+      this.customProperties = customProperties;
+    }
+
+    /**
+     * Setting the shards has to happen (in tests) after creating the collection because shards reference the collection
+     */
+    void setShards(Map<String, Shard> shards) {
+      this.shards = shards;
+    }
+
+    Set<String> getShardNames() {
+      return shards.keySet();
+    }
+
+    @Override
+    public String getName() {
+      return collectionName;
+    }
+
+    @Override
+    public Shard getShard(String name) {
+      return shards.get(name);
+    }
+
+    @Override
+    @Nonnull
+    public Iterator<Shard> iterator() {
+      return shards.values().iterator();
+    }
+
+    @Override
+    public Iterable<Shard> shards() {
+      return SolrCollectionImpl.this::iterator;
+    }
+
+    @Override
+    public String getCustomProperty(String customPropertyName) {
+      return customProperties.get(customPropertyName);
+    }
+  }
+
+
+  static class ShardImpl implements Shard {
+    private final String shardName;
+    private final SolrCollection collection;
+    private final ShardState shardState;
+    private Map<String, Replica> replicas;
+    private Replica leader;
+
+    ShardImpl(String shardName, SolrCollection collection, ShardState shardState) {
+      this.shardName = shardName;
+      this.collection = collection;
+      this.shardState = shardState;
+    }
+
+    /**
+     * Setting the replicas has to happen (in tests) after creating the shard because replicas reference the shard
+     */
+    void setReplicas(Map<String, Replica> replicas, Replica leader) {
+      this.replicas = replicas;
+      this.leader = leader;
+    }
+
+    @Override
+    public String getShardName() {
+      return shardName;
+    }
+
+    @Override
+    public SolrCollection getCollection() {
+      return collection;
+    }
+
+    @Override
+    public Replica getReplica(String name) {
+      return replicas.get(name);
+    }
+
+    @Override
+    @Nonnull
+    public Iterator<Replica> iterator() {
+      return replicas.values().iterator();
+    }
+
+    @Override
+    public Iterable<Replica> replicas() {
+      return ShardImpl.this::iterator;
+    }
+
+    @Override
+    public Replica getLeader() {
+      return leader;
+    }
+
+    @Override
+    public ShardState getState() {
+      return shardState;
+    }
+
+    public boolean equals(Object obj) {
+      if (obj == null) {
+        return false;
+      }
+      if (obj == this) {
+        return true;
+      }
+      if (obj.getClass() != getClass()) {
+        return false;
+      }
+      ShardImpl other = (ShardImpl) obj;
+      return Objects.equals(this.shardName, other.shardName)
+          && Objects.equals(this.collection, other.collection)
+          && Objects.equals(this.shardState, other.shardState)
+          && Objects.equals(this.replicas, other.replicas)
+          && Objects.equals(this.leader, other.leader);
+    }
+
+    public int hashCode() {
+      return Objects.hash(shardName, collection, shardState);
+    }
+  }
+
+
+  static class ReplicaImpl implements Replica {
+    private final String replicaName;
+    private final String coreName;
+    private final Shard shard;
+    private final ReplicaType replicaType;
+    private final ReplicaState replicaState;
+    private final Node node;
+
+    ReplicaImpl(String replicaName, String coreName, Shard shard, ReplicaType replicaType, ReplicaState replicaState, Node node) {
+      this.replicaName = replicaName;
+      this.coreName = coreName;
+      this.shard = shard;
+      this.replicaType = replicaType;
+      this.replicaState = replicaState;
+      this.node = node;
+    }
+
+    @Override
+    public Shard getShard() {
+      return shard;
+    }
+
+    @Override
+    public ReplicaType getType() {
+      return replicaType;
+    }
+
+    @Override
+    public ReplicaState getState() {
+      return replicaState;
+    }
+
+    @Override
+    public String getReplicaName() {
+      return replicaName;
+    }
+
+    @Override
+    public String getCoreName() {
+      return coreName;
+    }
+
+    @Override
+    public Node getNode() {
+      return node;
+    }
+
+    public boolean equals(Object obj) {
+      if (obj == null) {
+        return false;
+      }
+      if (obj == this) {
+        return true;
+      }
+      if (obj.getClass() != getClass()) {
+        return false;
+      }
+      ReplicaImpl other = (ReplicaImpl) obj;
+      return Objects.equals(this.replicaName, other.replicaName)
+          && Objects.equals(this.coreName, other.coreName)
+          && Objects.equals(this.shard, other.shard)
+          && Objects.equals(this.replicaType, other.replicaType)
+          && Objects.equals(this.replicaState, other.replicaState)
+          && Objects.equals(this.node, other.node);
+    }
+
+    public int hashCode() {
+      return Objects.hash(replicaName, coreName, shard, replicaType, replicaState, node);
     }
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
index 5b88b9d..61670da 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
@@ -29,66 +29,66 @@ import java.util.Set;
 
 public class PluginTestHelper {
 
-    static ClusterAbstractionsForTest.SolrCollectionImpl createCollection(String name, Map<String, String> properties,
-                                           int numShards, int nrtReplicas, int tlogReplicas, int pullReplicas, Set<Node> nodes) {
-        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(name, properties);
-        Map<String, Shard> shards = createShardsAndReplicas(solrCollection, numShards, nrtReplicas, tlogReplicas, pullReplicas, nodes);
-        solrCollection.setShards(shards);
-        return solrCollection;
-    }
+  static ClusterAbstractionsForTest.SolrCollectionImpl createCollection(String name, Map<String, String> properties,
+                                                                        int numShards, int nrtReplicas, int tlogReplicas, int pullReplicas, Set<Node> nodes) {
+    ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(name, properties);
+    Map<String, Shard> shards = createShardsAndReplicas(solrCollection, numShards, nrtReplicas, tlogReplicas, pullReplicas, nodes);
+    solrCollection.setShards(shards);
+    return solrCollection;
+  }
+
+  /**
+   * Builds the representation of shards for a collection, based on the number of shards and replicas for each to create.
+   * The replicas are allocated to the provided nodes in a round robin way. The leader is set to the last replica of each shard.
+   */
+  static Map<String, Shard> createShardsAndReplicas(SolrCollection collection, int numShards,
+                                                    int nrtReplicas, int tlogReplicas, int pullReplicas,
+                                                    Set<Node> nodes) {
+    Iterator<Node> nodeIterator = nodes.iterator();
+
+    Map<String, Shard> shards = new HashMap<>();
+
+    for (int s = 0; s < numShards; s++) {
+      // "traditional" shard name
+      String shardName = "shard" + (s + 1);
+
+      ClusterAbstractionsForTest.ShardImpl shard = new ClusterAbstractionsForTest.ShardImpl(shardName, collection, Shard.ShardState.ACTIVE);
+
+      Map<String, Replica> replicas = new HashMap<>();
+
+      Replica leader = null;
+      int totalReplicas = nrtReplicas + tlogReplicas + pullReplicas;
+      for (int r = 0; r < totalReplicas; r++) {
+        Replica.ReplicaType type;
+        if (r < nrtReplicas) {
+          type = Replica.ReplicaType.NRT;
+        } else if (r < nrtReplicas + tlogReplicas) {
+          type = Replica.ReplicaType.TLOG;
+        } else {
+          type = Replica.ReplicaType.PULL;
+        }
+        String replicaName = shardName + "_replica_" + type.getSuffixChar() + r;
+        String coreName = replicaName + "_c";
+        final Node node;
+        if (!nodeIterator.hasNext()) {
+          nodeIterator = nodes.iterator();
+        }
+        // If the nodes set is empty, this call will fail
+        node = nodeIterator.next();
 
-    /**
-     * Builds the representation of shards for a collection, based on the number of shards and replicas for each to create.
-     * The replicas are allocated to the provided nodes in a round robin way. The leader is set to the last replica of each shard.
-     */
-    static Map<String, Shard> createShardsAndReplicas(SolrCollection collection, int numShards,
-                                                      int nrtReplicas, int tlogReplicas, int pullReplicas,
-                                                      Set<Node> nodes) {
-        Iterator<Node> nodeIterator = nodes.iterator();
-
-        Map<String, Shard> shards = new HashMap<>();
-
-        for (int s = 0; s < numShards; s++) {
-            // "traditional" shard name
-            String shardName = "shard" + (s + 1);
-
-            ClusterAbstractionsForTest.ShardImpl shard = new ClusterAbstractionsForTest.ShardImpl(shardName, collection, Shard.ShardState.ACTIVE);
-
-            Map<String, Replica> replicas = new HashMap<>();
-
-            Replica leader = null;
-            int totalReplicas = nrtReplicas + tlogReplicas + pullReplicas;
-            for (int r = 0; r < totalReplicas; r++) {
-                Replica.ReplicaType type;
-                if (r < nrtReplicas) {
-                    type = Replica.ReplicaType.NRT;
-                } else if (r < nrtReplicas + tlogReplicas) {
-                    type = Replica.ReplicaType.TLOG;
-                } else {
-                    type = Replica.ReplicaType.PULL;
-                }
-                String replicaName = shardName + "_replica_" + type.getSuffixChar() + r;
-                String coreName = replicaName + "_c";
-                final Node node;
-                if (!nodeIterator.hasNext()) {
-                    nodeIterator = nodes.iterator();
-                }
-                // If the nodes set is empty, this call will fail
-                node = nodeIterator.next();
-
-                Replica replica = new ClusterAbstractionsForTest.ReplicaImpl(replicaName, coreName, shard, type, Replica.ReplicaState.ACTIVE, node);
-
-                replicas.put(replica.getReplicaName(), replica);
-                if (replica.getType() == Replica.ReplicaType.NRT) {
-                    leader = replica;
-                }
-            }
-
-            shard.setReplicas(replicas, leader);
-
-            shards.put(shard.getShardName(), shard);
+        Replica replica = new ClusterAbstractionsForTest.ReplicaImpl(replicaName, coreName, shard, type, Replica.ReplicaState.ACTIVE, node);
+
+        replicas.put(replica.getReplicaName(), replica);
+        if (replica.getType() == Replica.ReplicaType.NRT) {
+          leader = replica;
         }
+      }
+
+      shard.setReplicas(replicas, leader);
 
-        return shards;
+      shards.put(shard.getShardName(), shard);
     }
+
+    return shards;
+  }
 }

[lucene] 13/33: SOLR-15004: Test replica types placement.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 374fd7f79d222b205de804a987cd48c88eb8eed1
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue Nov 24 18:16:51 2020 +0100

    SOLR-15004: Test replica types placement.
---
 .../impl/AffinityPlacementFactoryTest.java         | 67 ++++++++++++++++++++++
 1 file changed, 67 insertions(+)

diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
index 67350d0..5a6911e 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
@@ -218,6 +218,73 @@ public class AffinityPlacementFactoryTest extends Assert {
     }
 
     @Test
+    public void testReplicaType() throws Exception {
+        String collectionName = "testCollection";
+        int NUM_NODES = 6;
+        Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(NUM_NODES);
+        for (int i = 0; i < NUM_NODES; i++) {
+            Builders.NodeBuilder nodeBuilder = clusterBuilder.getNodeBuilders().get(i);
+            nodeBuilder.setCoreCount(0);
+            nodeBuilder.setFreeDiskGB(100L);
+            if (i < NUM_NODES / 2) {
+                nodeBuilder.setSysprop(AffinityPlacementFactory.REPLICA_TYPE_SYSPROP, "Nrt,Tlog");
+                nodeBuilder.setSysprop("group", "one");
+            } else {
+                nodeBuilder.setSysprop(AffinityPlacementFactory.REPLICA_TYPE_SYSPROP, "Pull, foobar");
+                nodeBuilder.setSysprop("group", "two");
+            }
+        }
+
+        Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
+        collectionBuilder.initializeShardsReplicas(2, 0, 0, 0, clusterBuilder.getNodeBuilders());
+        clusterBuilder.addCollection(collectionBuilder);
+
+        Cluster cluster = clusterBuilder.build();
+
+        SolrCollection solrCollection = cluster.getCollection(collectionName);
+
+        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
+            StreamSupport.stream(solrCollection.shards().spliterator(), false)
+                .map(Shard::getShardName).collect(Collectors.toSet()),
+            cluster.getLiveNodes(), 2, 2, 2);
+
+        PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+        AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
+        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+        // 2 shards, 6 replicas
+        assertEquals(12, pp.getReplicaPlacements().size());
+        // shard -> group -> replica count
+        Map<Replica.ReplicaType, Map<String, Map<String, AtomicInteger>>> replicas = new HashMap<>();
+        AttributeValues attributeValues = attributeFetcher.fetchAttributes();
+        for (ReplicaPlacement rp : pp.getReplicaPlacements()) {
+            Optional<String> groupOptional = attributeValues.getSystemProperty(rp.getNode(), "group");
+            if (!groupOptional.isPresent()) {
+                fail("missing group sysprop for node " + rp.getNode());
+            }
+            String group = groupOptional.get();
+            if (group.equals("one")) {
+                assertTrue("wrong replica type in group one",
+                    (rp.getReplicaType() == Replica.ReplicaType.NRT) || rp.getReplicaType() == Replica.ReplicaType.TLOG);
+            } else {
+                assertEquals("wrong replica type in group two", Replica.ReplicaType.PULL, rp.getReplicaType());
+            }
+            replicas.computeIfAbsent(rp.getReplicaType(), type -> new HashMap<>())
+                .computeIfAbsent(rp.getShardName(), shard -> new HashMap<>())
+                .computeIfAbsent(group, g -> new AtomicInteger()).incrementAndGet();
+        }
+        replicas.forEach((type, perTypeReplicas) -> {
+            perTypeReplicas.forEach((shard, groupCounts) -> {
+                assertEquals("number of groups", 1, groupCounts.size());
+                groupCounts.forEach((group, count) -> {
+                    assertTrue("too few replicas shard=" + shard + ", type=" + type + ", group=" + group,
+                        count.get() >= 1);
+                });
+            });
+        });
+
+    }
+
+    @Test
     //@Ignore
     public void testScalability() throws Exception {
         log.info("==== numNodes ====");

[lucene] 20/33: Reformat placement plugin code to right tab size

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 8450ea8ce78d2b89421dedf9d77e749baeeda4b2
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Thu Nov 26 11:05:52 2020 +0100

    Reformat placement plugin code to right tab size
---
 .../solr/cluster/placement/AttributeFetcher.java   |  49 ++-
 .../solr/cluster/placement/AttributeValues.java    |  60 ++--
 .../solr/cluster/placement/PlacementPlan.java      |   2 +-
 .../solr/cluster/placement/PlacementPlugin.java    |  14 +-
 .../cluster/placement/PlacementPluginConfig.java   |   4 +-
 .../solr/cluster/placement/PlacementRequest.java   |  54 +--
 .../placement/impl/AttributeFetcherImpl.java       | 376 ++++++++++-----------
 .../placement/impl/AttributeValuesImpl.java        | 146 ++++----
 .../placement/impl/PlacementPlanFactoryImpl.java   |  16 +-
 .../placement/impl/PlacementPluginConfigImpl.java  |  10 +-
 .../placement/impl/PlacementRequestImpl.java       |   4 +-
 .../impl/SimpleClusterAbstractionsImpl.java        | 115 ++++---
 .../plugins/AffinityPlacementFactory.java          |  34 +-
 13 files changed, 487 insertions(+), 397 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/AttributeFetcher.java b/solr/core/src/java/org/apache/solr/cluster/placement/AttributeFetcher.java
index cb368d7..9578326 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/AttributeFetcher.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/AttributeFetcher.java
@@ -25,31 +25,49 @@ import java.util.Set;
  * <p>Instances of this interface are used to fetch various attributes from nodes (and other sources) in the cluster.</p>
  */
 public interface AttributeFetcher {
-  /** Request the number of cores on each node. To get the value use {@link AttributeValues#getCoresCount(Node)} */
+  /**
+   * Request the number of cores on each node. To get the value use {@link AttributeValues#getCoresCount(Node)}
+   */
   AttributeFetcher requestNodeCoreCount();
 
-  /** Request the disk hardware type on each node. To get the value use {@link AttributeValues#getDiskType(Node)} */
+  /**
+   * Request the disk hardware type on each node. To get the value use {@link AttributeValues#getDiskType(Node)}
+   */
   AttributeFetcher requestNodeDiskType();
 
-  /** Request the free disk size on each node. To get the value use {@link AttributeValues#getFreeDisk(Node)} */
+  /**
+   * Request the free disk size on each node. To get the value use {@link AttributeValues#getFreeDisk(Node)}
+   */
   AttributeFetcher requestNodeFreeDisk();
 
-  /** Request the total disk size on each node. To get the value use {@link AttributeValues#getTotalDisk(Node)} */
+  /**
+   * Request the total disk size on each node. To get the value use {@link AttributeValues#getTotalDisk(Node)}
+   */
   AttributeFetcher requestNodeTotalDisk();
 
-  /** Request the heap usage on each node. To get the value use {@link AttributeValues#getHeapUsage(Node)} */
+  /**
+   * Request the heap usage on each node. To get the value use {@link AttributeValues#getHeapUsage(Node)}
+   */
   AttributeFetcher requestNodeHeapUsage();
 
-  /** Request the system load average on each node. To get the value use {@link AttributeValues#getSystemLoadAverage(Node)} */
+  /**
+   * Request the system load average on each node. To get the value use {@link AttributeValues#getSystemLoadAverage(Node)}
+   */
   AttributeFetcher requestNodeSystemLoadAverage();
 
-  /** Request a given system property on each node. To get the value use {@link AttributeValues#getSystemProperty(Node, String)} */
+  /**
+   * Request a given system property on each node. To get the value use {@link AttributeValues#getSystemProperty(Node, String)}
+   */
   AttributeFetcher requestNodeSystemProperty(String name);
 
-  /** Request an environment variable on each node. To get the value use {@link AttributeValues#getEnvironmentVariable(Node, String)} */
+  /**
+   * Request an environment variable on each node. To get the value use {@link AttributeValues#getEnvironmentVariable(Node, String)}
+   */
   AttributeFetcher requestNodeEnvironmentVariable(String name);
 
-  /** Request a node metric from each node. To get the value use {@link AttributeValues#getMetric(Node, String, NodeMetricRegistry)} */
+  /**
+   * Request a node metric from each node. To get the value use {@link AttributeValues#getMetric(Node, String, NodeMetricRegistry)}
+   */
   AttributeFetcher requestNodeMetric(String metricName, NodeMetricRegistry registry);
 
 
@@ -59,12 +77,15 @@ public interface AttributeFetcher {
    */
   AttributeFetcher fetchFrom(Set<Node> nodes);
 
-  /** Requests a (non node) metric of a given scope and name. To get the value use {@link AttributeValues#getMetric(String, String)} */
+  /**
+   * Requests a (non node) metric of a given scope and name. To get the value use {@link AttributeValues#getMetric(String, String)}
+   */
   AttributeFetcher requestMetric(String scope, String metricName);
 
   /**
    * Fetches all requested node attributes from all nodes passed to {@link #fetchFrom(Set)} as well as non node attributes
    * (those requested for example using {@link #requestMetric(String, String)}.
+   *
    * @return An instance allowing retrieval of all attributed that could be fetched.
    */
   AttributeValues fetchAttributes();
@@ -73,9 +94,13 @@ public interface AttributeFetcher {
    * Registry options for {@link Node} metrics.
    */
   enum NodeMetricRegistry {
-    /** corresponds to solr.node */
+    /**
+     * corresponds to solr.node
+     */
     SOLR_NODE,
-    /** corresponds to solr.jvm */
+    /**
+     * corresponds to solr.jvm
+     */
     SOLR_JVM
   }
 
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/AttributeValues.java b/solr/core/src/java/org/apache/solr/cluster/placement/AttributeValues.java
index 4519c8a..24fcb6f 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/AttributeValues.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/AttributeValues.java
@@ -22,34 +22,54 @@ import org.apache.solr.cluster.Node;
 import java.util.Optional;
 
 public interface AttributeValues {
-    /** For the given node: number of cores */
-    Optional<Integer> getCoresCount(Node node);
+  /**
+   * For the given node: number of cores
+   */
+  Optional<Integer> getCoresCount(Node node);
 
-    /** For the given node: Hardware type of the disk partition where cores are stored */
-    Optional<AttributeFetcher.DiskHardwareType> getDiskType(Node node);
+  /**
+   * For the given node: Hardware type of the disk partition where cores are stored
+   */
+  Optional<AttributeFetcher.DiskHardwareType> getDiskType(Node node);
 
-    /** For the given node: Free disk size in Gigabytes of the partition on which cores are stored */
-    Optional<Long> getFreeDisk(Node node);
+  /**
+   * For the given node: Free disk size in Gigabytes of the partition on which cores are stored
+   */
+  Optional<Long> getFreeDisk(Node node);
 
-    /** For the given node: Total disk size in Gigabytes of the partition on which cores are stored */
-    Optional<Long> getTotalDisk(Node node);
+  /**
+   * For the given node: Total disk size in Gigabytes of the partition on which cores are stored
+   */
+  Optional<Long> getTotalDisk(Node node);
 
-    /** For the given node: Percentage between 0 and 100 of used heap over max heap */
-    Optional<Double> getHeapUsage(Node node);
+  /**
+   * For the given node: Percentage between 0 and 100 of used heap over max heap
+   */
+  Optional<Double> getHeapUsage(Node node);
 
-    /** For the given node: matches {@link java.lang.management.OperatingSystemMXBean#getSystemLoadAverage()} */
-    Optional<Double> getSystemLoadAverage(Node node);
+  /**
+   * For the given node: matches {@link java.lang.management.OperatingSystemMXBean#getSystemLoadAverage()}
+   */
+  Optional<Double> getSystemLoadAverage(Node node);
 
-    /** For the given node: system property value (system properties are passed to Java using {@code -Dname=value} */
-    Optional<String> getSystemProperty(Node node, String name);
+  /**
+   * For the given node: system property value (system properties are passed to Java using {@code -Dname=value}
+   */
+  Optional<String> getSystemProperty(Node node, String name);
 
-    /** For the given node: environment variable value */
-    Optional<String> getEnvironmentVariable(Node node, String name);
+  /**
+   * For the given node: environment variable value
+   */
+  Optional<String> getEnvironmentVariable(Node node, String name);
 
-    /** For the given node: metric of specific name and registry */
-    Optional<Double> getMetric(Node node, String metricName, AttributeFetcher.NodeMetricRegistry registry);
+  /**
+   * For the given node: metric of specific name and registry
+   */
+  Optional<Double> getMetric(Node node, String metricName, AttributeFetcher.NodeMetricRegistry registry);
 
 
-    /** Get a non node related metric of specific scope and name */
-    Optional<Double> getMetric(String scope, String metricName);
+  /**
+   * Get a non node related metric of specific scope and name
+   */
+  Optional<Double> getMetric(String scope, String metricName);
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlan.java b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlan.java
index c4738a5..331578b 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlan.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlan.java
@@ -24,7 +24,7 @@ import java.util.Set;
 /**
  * A fully specified plan or instructions for placement, deletion or move to be applied to the cluster.<p>
  * Fully specified means the actual {@link Node}'s on which to place replicas have been decided.
- *
+ * <p>
  * Instances are created by plugin code using {@link PlacementPlanFactory}. This interface obviously doesn't expose much but
  * the underlying Solr side implementation has all that is needed (and will do at least one cast in order to execute the
  * plan, likely then using some type of visitor pattern).
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 28b6476..bbb52cb 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
@@ -36,13 +36,13 @@ public interface PlacementPlugin {
    *
    * <p>Configuration is passed upon creation of a new instance of this class by {@link PlacementPluginFactory#createPluginInstance}.
    *
-   * @param cluster initial state of the cluster. Note there are {@link java.util.Set}'s and {@link java.util.Map}'s
-   *                accessible from the {@link Cluster} and other reachable instances. These collection will not change
-   *                while the plugin is executing and will be thrown away once the plugin is done. The plugin code can
-   *                therefore modify them if needed.
-   * @param placementRequest request for placing new replicas or moving existing replicas on the cluster.
-   * @param attributeFetcher Factory used by the plugin to fetch additional attributes from the cluster nodes, such as
-   *                         count of coresm ssytem properties etc..
+   * @param cluster              initial state of the cluster. Note there are {@link java.util.Set}'s and {@link java.util.Map}'s
+   *                             accessible from the {@link Cluster} and other reachable instances. These collection will not change
+   *                             while the plugin is executing and will be thrown away once the plugin is done. The plugin code can
+   *                             therefore modify them if needed.
+   * @param placementRequest     request for placing new replicas or moving existing replicas on the cluster.
+   * @param attributeFetcher     Factory used by the plugin to fetch additional attributes from the cluster nodes, such as
+   *                             count of coresm ssytem properties etc..
    * @param placementPlanFactory Factory used to create instances of {@link PlacementPlan} to return computed decision.
    * @return plan satisfying the placement request.
    */
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 4f17051..d223dcc 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
@@ -73,7 +73,9 @@ 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 */
+  /**
+   * Name of the property containing the factory class
+   */
   String FACTORY_CLASS = "class";
 
   /**
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementRequest.java b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementRequest.java
index 61b49dd..44222a2 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/PlacementRequest.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/PlacementRequest.java
@@ -31,34 +31,34 @@ import java.util.Set;
  * is specified (defaults to being equal to the set returned by {@link Cluster#getLiveNodes()}).
  */
 public interface PlacementRequest {
-    /**
-     * The {@link SolrCollection} to add {@link Replica}(s) to.
-     */
-    SolrCollection getCollection();
+  /**
+   * The {@link SolrCollection} to add {@link Replica}(s) to.
+   */
+  SolrCollection getCollection();
 
-    /**
-     * <p>Shard name(s) for which new replicas placement should be computed. The shard(s) might exist or not (that's why this
-     * method returns a {@link Set} of {@link String}'s and not directly a set of {@link Shard} instances).
-     *
-     * <p>Note the Collection API allows specifying the shard name or a {@code _route_} parameter. The Solr implementation will
-     * convert either specification into the relevant shard name so the plugin code doesn't have to worry about this.
-     */
-    Set<String> getShardNames();
+  /**
+   * <p>Shard name(s) for which new replicas placement should be computed. The shard(s) might exist or not (that's why this
+   * method returns a {@link Set} of {@link String}'s and not directly a set of {@link Shard} instances).
+   *
+   * <p>Note the Collection API allows specifying the shard name or a {@code _route_} parameter. The Solr implementation will
+   * convert either specification into the relevant shard name so the plugin code doesn't have to worry about this.
+   */
+  Set<String> getShardNames();
 
-    /**
-     * <p>Replicas should only be placed on nodes in the set returned by this method.
-     *
-     * <p>When Collection API calls do not specify a specific set of target nodes, replicas can be placed on any live node of
-     * the cluster. In such cases, this set will be equal to the set of all live nodes. The plugin placement code does not
-     * need to worry (or care) if a set of nodes was explicitly specified or not.
-     *
-     * @return never {@code null} and never empty set (if that set was to be empty for any reason, no placement would be
-     * possible and the Solr infrastructure driving the plugin code would detect the error itself rather than calling the plugin).
-     */
-    Set<Node> getTargetNodes();
+  /**
+   * <p>Replicas should only be placed on nodes in the set returned by this method.
+   *
+   * <p>When Collection API calls do not specify a specific set of target nodes, replicas can be placed on any live node of
+   * the cluster. In such cases, this set will be equal to the set of all live nodes. The plugin placement code does not
+   * need to worry (or care) if a set of nodes was explicitly specified or not.
+   *
+   * @return never {@code null} and never empty set (if that set was to be empty for any reason, no placement would be
+   * possible and the Solr infrastructure driving the plugin code would detect the error itself rather than calling the plugin).
+   */
+  Set<Node> getTargetNodes();
 
-    /**
-     * Returns the number of replica to create for the given replica type.
-     */
-    int getCountReplicasToCreate(Replica.ReplicaType replicaType);
+  /**
+   * Returns the number of replica to create for the given replica type.
+   */
+  int getCountReplicasToCreate(Replica.ReplicaType replicaType);
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeFetcherImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeFetcherImpl.java
index 97d4ba1..3c3bf49 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeFetcherImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeFetcherImpl.java
@@ -35,194 +35,194 @@ import java.util.*;
 import java.util.function.BiConsumer;
 
 public class AttributeFetcherImpl implements AttributeFetcher {
-    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-    boolean requestedNodeCoreCount;
-    boolean requestedNodeDiskType;
-    boolean requestedNodeFreeDisk;
-    boolean requestedNodeTotalDisk;
-    boolean requestedNodeHeapUsage;
-    boolean requestedNodeSystemLoadAverage;
-    Set<String> requestedNodeSystemPropertiesSnitchTags = new HashSet<>();
-    Set<String> requestedNodeMetricSnitchTags  = new HashSet<>();
-
-    Set<Node> nodes = Collections.emptySet();
-
-    private final SolrCloudManager cloudManager;
-
-    AttributeFetcherImpl(SolrCloudManager cloudManager) {
-        this.cloudManager = cloudManager;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeCoreCount() {
-        requestedNodeCoreCount = true;
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeDiskType() {
-        requestedNodeDiskType = true;
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeFreeDisk() {
-        requestedNodeFreeDisk = true;
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeTotalDisk() {
-        requestedNodeTotalDisk = true;
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeHeapUsage() {
-        requestedNodeHeapUsage = true;
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeSystemLoadAverage() {
-        requestedNodeSystemLoadAverage = true;
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeSystemProperty(String name) {
-        requestedNodeSystemPropertiesSnitchTags.add(getSystemPropertySnitchTag(name));
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestNodeEnvironmentVariable(String name) {
-        throw new UnsupportedOperationException("Not yet implemented...");
-    }
-
-    @Override
-    public AttributeFetcher requestNodeMetric(String metricName, NodeMetricRegistry registry) {
-        requestedNodeMetricSnitchTags.add(getMetricSnitchTag(metricName, registry));
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher fetchFrom(Set<Node> nodes) {
-        this.nodes = nodes;
-        return this;
-    }
-
-    @Override
-    public AttributeFetcher requestMetric(String scope, String metricName) {
-        throw new UnsupportedOperationException("Not yet implemented...");
-    }
-
-    @Override
-    public AttributeValues fetchAttributes() {
-        // TODO Code here only supports node related attributes for now
-
-        // Maps in which attribute values will be added
-        Map<Node, Integer> nodeToCoreCount = Maps.newHashMap();
-        Map<Node, DiskHardwareType> nodeToDiskType = Maps.newHashMap();
-        Map<Node, Long> nodeToFreeDisk = Maps.newHashMap();
-        Map<Node, Long> nodeToTotalDisk = Maps.newHashMap();
-        Map<Node, Double> nodeToHeapUsage = Maps.newHashMap();
-        Map<Node, Double> nodeToSystemLoadAverage = Maps.newHashMap();
-        Map<String, Map<Node, String>> syspropSnitchToNodeToValue = Maps.newHashMap();
-        Map<String, Map<Node, Double>> metricSnitchToNodeToValue = Maps.newHashMap();
-
-        // In order to match the returned values for the various snitches, we need to keep track of where each
-        // received value goes. Given the target maps are of different types (the maps from Node to whatever defined
-        // above) we instead pass a function taking two arguments, the node and the (non null) returned value,
-        // that will cast the value into the appropriate type for the snitch tag and insert it into the appropriate map
-        // with the node as the key.
-        Map<String, BiConsumer<Node, Object>> allSnitchTagsToInsertion = Maps.newHashMap();
-        if (requestedNodeCoreCount) {
-            allSnitchTagsToInsertion.put(ImplicitSnitch.CORES, (node, value) -> nodeToCoreCount.put(node, ((Number) value).intValue()));
-        }
-        if (requestedNodeDiskType) {
-            allSnitchTagsToInsertion.put(ImplicitSnitch.DISKTYPE, (node, value) -> {
-                if ("rotational".equals(value)) {
-                    nodeToDiskType.put(node, DiskHardwareType.ROTATIONAL);
-                } else if ("ssd".equals(value)) {
-                    nodeToDiskType.put(node, DiskHardwareType.SSD);
-                }
-                // unknown disk type: insert no value, returned optional will be empty
-            });
-        }
-        if (requestedNodeFreeDisk) {
-            allSnitchTagsToInsertion.put(SolrClientNodeStateProvider.Variable.FREEDISK.tagName,
-                    // Convert from bytes to GB
-                    (node, value) -> nodeToFreeDisk.put(node, ((Number) value).longValue() / 1024 / 1024 / 1024));
-        }
-        if (requestedNodeTotalDisk) {
-            allSnitchTagsToInsertion.put(SolrClientNodeStateProvider.Variable.TOTALDISK.tagName,
-                    // Convert from bytes to GB
-                    (node, value) -> nodeToTotalDisk.put(node, ((Number) value).longValue() / 1024 / 1024 / 1024));
-        }
-        if (requestedNodeHeapUsage) {
-            allSnitchTagsToInsertion.put(ImplicitSnitch.HEAPUSAGE,
-                    (node, value) -> nodeToHeapUsage.put(node, ((Number) value).doubleValue()));
-        }
-        if (requestedNodeSystemLoadAverage) {
-            allSnitchTagsToInsertion.put(ImplicitSnitch.SYSLOADAVG,
-                    (node, value) -> nodeToSystemLoadAverage.put(node, ((Number) value).doubleValue()));
-        }
-        for (String sysPropSnitch : requestedNodeSystemPropertiesSnitchTags) {
-            final Map<Node, String> sysPropMap = Maps.newHashMap();
-            syspropSnitchToNodeToValue.put(sysPropSnitch, sysPropMap);
-            allSnitchTagsToInsertion.put(sysPropSnitch, (node, value) -> sysPropMap.put(node, (String) value));
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  boolean requestedNodeCoreCount;
+  boolean requestedNodeDiskType;
+  boolean requestedNodeFreeDisk;
+  boolean requestedNodeTotalDisk;
+  boolean requestedNodeHeapUsage;
+  boolean requestedNodeSystemLoadAverage;
+  Set<String> requestedNodeSystemPropertiesSnitchTags = new HashSet<>();
+  Set<String> requestedNodeMetricSnitchTags = new HashSet<>();
+
+  Set<Node> nodes = Collections.emptySet();
+
+  private final SolrCloudManager cloudManager;
+
+  AttributeFetcherImpl(SolrCloudManager cloudManager) {
+    this.cloudManager = cloudManager;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeCoreCount() {
+    requestedNodeCoreCount = true;
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeDiskType() {
+    requestedNodeDiskType = true;
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeFreeDisk() {
+    requestedNodeFreeDisk = true;
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeTotalDisk() {
+    requestedNodeTotalDisk = true;
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeHeapUsage() {
+    requestedNodeHeapUsage = true;
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeSystemLoadAverage() {
+    requestedNodeSystemLoadAverage = true;
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeSystemProperty(String name) {
+    requestedNodeSystemPropertiesSnitchTags.add(getSystemPropertySnitchTag(name));
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestNodeEnvironmentVariable(String name) {
+    throw new UnsupportedOperationException("Not yet implemented...");
+  }
+
+  @Override
+  public AttributeFetcher requestNodeMetric(String metricName, NodeMetricRegistry registry) {
+    requestedNodeMetricSnitchTags.add(getMetricSnitchTag(metricName, registry));
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher fetchFrom(Set<Node> nodes) {
+    this.nodes = nodes;
+    return this;
+  }
+
+  @Override
+  public AttributeFetcher requestMetric(String scope, String metricName) {
+    throw new UnsupportedOperationException("Not yet implemented...");
+  }
+
+  @Override
+  public AttributeValues fetchAttributes() {
+    // TODO Code here only supports node related attributes for now
+
+    // Maps in which attribute values will be added
+    Map<Node, Integer> nodeToCoreCount = Maps.newHashMap();
+    Map<Node, DiskHardwareType> nodeToDiskType = Maps.newHashMap();
+    Map<Node, Long> nodeToFreeDisk = Maps.newHashMap();
+    Map<Node, Long> nodeToTotalDisk = Maps.newHashMap();
+    Map<Node, Double> nodeToHeapUsage = Maps.newHashMap();
+    Map<Node, Double> nodeToSystemLoadAverage = Maps.newHashMap();
+    Map<String, Map<Node, String>> syspropSnitchToNodeToValue = Maps.newHashMap();
+    Map<String, Map<Node, Double>> metricSnitchToNodeToValue = Maps.newHashMap();
+
+    // In order to match the returned values for the various snitches, we need to keep track of where each
+    // received value goes. Given the target maps are of different types (the maps from Node to whatever defined
+    // above) we instead pass a function taking two arguments, the node and the (non null) returned value,
+    // that will cast the value into the appropriate type for the snitch tag and insert it into the appropriate map
+    // with the node as the key.
+    Map<String, BiConsumer<Node, Object>> allSnitchTagsToInsertion = Maps.newHashMap();
+    if (requestedNodeCoreCount) {
+      allSnitchTagsToInsertion.put(ImplicitSnitch.CORES, (node, value) -> nodeToCoreCount.put(node, ((Number) value).intValue()));
+    }
+    if (requestedNodeDiskType) {
+      allSnitchTagsToInsertion.put(ImplicitSnitch.DISKTYPE, (node, value) -> {
+        if ("rotational".equals(value)) {
+          nodeToDiskType.put(node, DiskHardwareType.ROTATIONAL);
+        } else if ("ssd".equals(value)) {
+          nodeToDiskType.put(node, DiskHardwareType.SSD);
         }
-        for (String metricSnitch : requestedNodeMetricSnitchTags) {
-            final Map<Node, Double> metricMap = Maps.newHashMap();
-            metricSnitchToNodeToValue.put(metricSnitch, metricMap);
-            allSnitchTagsToInsertion.put(metricSnitch, (node, value) -> metricMap.put(node, (Double) value));
+        // unknown disk type: insert no value, returned optional will be empty
+      });
+    }
+    if (requestedNodeFreeDisk) {
+      allSnitchTagsToInsertion.put(SolrClientNodeStateProvider.Variable.FREEDISK.tagName,
+          // Convert from bytes to GB
+          (node, value) -> nodeToFreeDisk.put(node, ((Number) value).longValue() / 1024 / 1024 / 1024));
+    }
+    if (requestedNodeTotalDisk) {
+      allSnitchTagsToInsertion.put(SolrClientNodeStateProvider.Variable.TOTALDISK.tagName,
+          // Convert from bytes to GB
+          (node, value) -> nodeToTotalDisk.put(node, ((Number) value).longValue() / 1024 / 1024 / 1024));
+    }
+    if (requestedNodeHeapUsage) {
+      allSnitchTagsToInsertion.put(ImplicitSnitch.HEAPUSAGE,
+          (node, value) -> nodeToHeapUsage.put(node, ((Number) value).doubleValue()));
+    }
+    if (requestedNodeSystemLoadAverage) {
+      allSnitchTagsToInsertion.put(ImplicitSnitch.SYSLOADAVG,
+          (node, value) -> nodeToSystemLoadAverage.put(node, ((Number) value).doubleValue()));
+    }
+    for (String sysPropSnitch : requestedNodeSystemPropertiesSnitchTags) {
+      final Map<Node, String> sysPropMap = Maps.newHashMap();
+      syspropSnitchToNodeToValue.put(sysPropSnitch, sysPropMap);
+      allSnitchTagsToInsertion.put(sysPropSnitch, (node, value) -> sysPropMap.put(node, (String) value));
+    }
+    for (String metricSnitch : requestedNodeMetricSnitchTags) {
+      final Map<Node, Double> metricMap = Maps.newHashMap();
+      metricSnitchToNodeToValue.put(metricSnitch, metricMap);
+      allSnitchTagsToInsertion.put(metricSnitch, (node, value) -> metricMap.put(node, (Double) value));
+    }
+
+    // Now that we know everything we need to fetch (and where to put it), just do it.
+    for (Node node : nodes) {
+      Map<String, Object> tagValues = cloudManager.getNodeStateProvider().getNodeValues(node.getName(), allSnitchTagsToInsertion.keySet());
+      for (Map.Entry<String, Object> e : tagValues.entrySet()) {
+        String tag = e.getKey();
+        Object value = e.getValue(); // returned value from the node
+
+        BiConsumer<Node, Object> inserter = allSnitchTagsToInsertion.get(tag);
+        // If inserter is null it's a return of a tag that we didn't request
+        if (inserter != null) {
+          inserter.accept(node, value);
+        } else {
+          log.error("Received unsolicited snitch tag {} from node {}", tag, node);
         }
-
-        // Now that we know everything we need to fetch (and where to put it), just do it.
-        for (Node node : nodes) {
-            Map<String, Object> tagValues = cloudManager.getNodeStateProvider().getNodeValues(node.getName(), allSnitchTagsToInsertion.keySet());
-            for (Map.Entry<String, Object> e : tagValues.entrySet()) {
-                String tag = e.getKey();
-                Object value = e.getValue(); // returned value from the node
-
-                BiConsumer<Node, Object> inserter = allSnitchTagsToInsertion.get(tag);
-                // If inserter is null it's a return of a tag that we didn't request
-                if (inserter != null) {
-                    inserter.accept(node, value);
-                } else {
-                    log.error("Received unsolicited snitch tag {} from node {}", tag, node);
-                }
-            }
-        }
-
-        return new AttributeValuesImpl(nodeToCoreCount,
-                                       nodeToDiskType,
-                                       nodeToFreeDisk,
-                                       nodeToTotalDisk,
-                                       nodeToHeapUsage,
-                                       nodeToSystemLoadAverage,
-                                       syspropSnitchToNodeToValue,
-                                       metricSnitchToNodeToValue);
-    }
-
-    private static SolrInfoBean.Group getGroupFromMetricRegistry(NodeMetricRegistry registry) {
-        switch (registry) {
-            case SOLR_JVM:
-                return SolrInfoBean.Group.jvm;
-            case SOLR_NODE:
-                return SolrInfoBean.Group.node;
-            default:
-                throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unsupported registry value " + registry);
-        }
-    }
-
-    public static String getMetricSnitchTag(String metricName, NodeMetricRegistry registry) {
-        return SolrClientNodeStateProvider.METRICS_PREFIX + SolrMetricManager.getRegistryName(getGroupFromMetricRegistry(registry), metricName);
-    }
-
-    public static String getSystemPropertySnitchTag(String name) {
-        return ImplicitSnitch.SYSPROP + name;
-    }
+      }
+    }
+
+    return new AttributeValuesImpl(nodeToCoreCount,
+        nodeToDiskType,
+        nodeToFreeDisk,
+        nodeToTotalDisk,
+        nodeToHeapUsage,
+        nodeToSystemLoadAverage,
+        syspropSnitchToNodeToValue,
+        metricSnitchToNodeToValue);
+  }
+
+  private static SolrInfoBean.Group getGroupFromMetricRegistry(NodeMetricRegistry registry) {
+    switch (registry) {
+      case SOLR_JVM:
+        return SolrInfoBean.Group.jvm;
+      case SOLR_NODE:
+        return SolrInfoBean.Group.node;
+      default:
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unsupported registry value " + registry);
+    }
+  }
+
+  public static String getMetricSnitchTag(String metricName, NodeMetricRegistry registry) {
+    return SolrClientNodeStateProvider.METRICS_PREFIX + SolrMetricManager.getRegistryName(getGroupFromMetricRegistry(registry), metricName);
+  }
+
+  public static String getSystemPropertySnitchTag(String name) {
+    return ImplicitSnitch.SYSPROP + name;
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeValuesImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeValuesImpl.java
index 0f4c1ab..ce68094 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeValuesImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/AttributeValuesImpl.java
@@ -25,90 +25,90 @@ import java.util.Map;
 import java.util.Optional;
 
 public class AttributeValuesImpl implements AttributeValues {
-    final Map<Node, Integer> nodeToCoreCount;
-    final Map<Node, AttributeFetcher.DiskHardwareType> nodeToDiskType;
-    final Map<Node, Long> nodeToFreeDisk;
-    final Map<Node, Long> nodeToTotalDisk;
-    final Map<Node, Double> nodeToHeapUsage;
-    final Map<Node, Double> nodeToSystemLoadAverage;
-    final Map<String, Map<Node, String>> syspropSnitchToNodeToValue;
-    final Map<String, Map<Node, Double>> metricSnitchToNodeToValue;
+  final Map<Node, Integer> nodeToCoreCount;
+  final Map<Node, AttributeFetcher.DiskHardwareType> nodeToDiskType;
+  final Map<Node, Long> nodeToFreeDisk;
+  final Map<Node, Long> nodeToTotalDisk;
+  final Map<Node, Double> nodeToHeapUsage;
+  final Map<Node, Double> nodeToSystemLoadAverage;
+  final Map<String, Map<Node, String>> syspropSnitchToNodeToValue;
+  final Map<String, Map<Node, Double>> metricSnitchToNodeToValue;
 
-    public AttributeValuesImpl(Map<Node, Integer> nodeToCoreCount,
-                        Map<Node, AttributeFetcher.DiskHardwareType> nodeToDiskType,
-                        Map<Node, Long> nodeToFreeDisk,
-                        Map<Node, Long> nodeToTotalDisk,
-                        Map<Node, Double> nodeToHeapUsage,
-                        Map<Node, Double> nodeToSystemLoadAverage,
-                        Map<String, Map<Node, String>> syspropSnitchToNodeToValue,
-                        Map<String, Map<Node, Double>> metricSnitchToNodeToValue) {
-        this.nodeToCoreCount = nodeToCoreCount;
-        this.nodeToDiskType = nodeToDiskType;
-        this.nodeToFreeDisk = nodeToFreeDisk;
-        this.nodeToTotalDisk = nodeToTotalDisk;
-        this.nodeToHeapUsage = nodeToHeapUsage;
-        this.nodeToSystemLoadAverage = nodeToSystemLoadAverage;
-        this.syspropSnitchToNodeToValue = syspropSnitchToNodeToValue;
-        this.metricSnitchToNodeToValue = metricSnitchToNodeToValue;
-    }
+  public AttributeValuesImpl(Map<Node, Integer> nodeToCoreCount,
+                             Map<Node, AttributeFetcher.DiskHardwareType> nodeToDiskType,
+                             Map<Node, Long> nodeToFreeDisk,
+                             Map<Node, Long> nodeToTotalDisk,
+                             Map<Node, Double> nodeToHeapUsage,
+                             Map<Node, Double> nodeToSystemLoadAverage,
+                             Map<String, Map<Node, String>> syspropSnitchToNodeToValue,
+                             Map<String, Map<Node, Double>> metricSnitchToNodeToValue) {
+    this.nodeToCoreCount = nodeToCoreCount;
+    this.nodeToDiskType = nodeToDiskType;
+    this.nodeToFreeDisk = nodeToFreeDisk;
+    this.nodeToTotalDisk = nodeToTotalDisk;
+    this.nodeToHeapUsage = nodeToHeapUsage;
+    this.nodeToSystemLoadAverage = nodeToSystemLoadAverage;
+    this.syspropSnitchToNodeToValue = syspropSnitchToNodeToValue;
+    this.metricSnitchToNodeToValue = metricSnitchToNodeToValue;
+  }
 
-    @Override
-    public Optional<Integer> getCoresCount(Node node) {
-        return Optional.ofNullable(nodeToCoreCount.get(node));
-    }
+  @Override
+  public Optional<Integer> getCoresCount(Node node) {
+    return Optional.ofNullable(nodeToCoreCount.get(node));
+  }
 
-    @Override
-    public Optional<AttributeFetcher.DiskHardwareType> getDiskType(Node node) {
-        return Optional.ofNullable(nodeToDiskType.get(node));
-    }
+  @Override
+  public Optional<AttributeFetcher.DiskHardwareType> getDiskType(Node node) {
+    return Optional.ofNullable(nodeToDiskType.get(node));
+  }
 
-    @Override
-    public Optional<Long> getFreeDisk(Node node) {
-        return Optional.ofNullable(nodeToFreeDisk.get(node));
-    }
+  @Override
+  public Optional<Long> getFreeDisk(Node node) {
+    return Optional.ofNullable(nodeToFreeDisk.get(node));
+  }
 
-    @Override
-    public Optional<Long> getTotalDisk(Node node) {
-        return Optional.ofNullable(nodeToTotalDisk.get(node));
-    }
+  @Override
+  public Optional<Long> getTotalDisk(Node node) {
+    return Optional.ofNullable(nodeToTotalDisk.get(node));
+  }
 
-    @Override
-    public Optional<Double> getHeapUsage(Node node) {
-        return Optional.ofNullable(nodeToHeapUsage.get(node));
-    }
+  @Override
+  public Optional<Double> getHeapUsage(Node node) {
+    return Optional.ofNullable(nodeToHeapUsage.get(node));
+  }
 
-    @Override
-    public Optional<Double> getSystemLoadAverage(Node node) {
-        return Optional.ofNullable(nodeToSystemLoadAverage.get(node));
-    }
+  @Override
+  public Optional<Double> getSystemLoadAverage(Node node) {
+    return Optional.ofNullable(nodeToSystemLoadAverage.get(node));
+  }
 
-    @Override
-    public Optional<String> getSystemProperty(Node node, String name) {
-        Map<Node, String> nodeToValue = syspropSnitchToNodeToValue.get(AttributeFetcherImpl.getSystemPropertySnitchTag(name));
-        if (nodeToValue == null) {
-            return Optional.empty();
-        }
-        return Optional.ofNullable(nodeToValue.get(node));
+  @Override
+  public Optional<String> getSystemProperty(Node node, String name) {
+    Map<Node, String> nodeToValue = syspropSnitchToNodeToValue.get(AttributeFetcherImpl.getSystemPropertySnitchTag(name));
+    if (nodeToValue == null) {
+      return Optional.empty();
     }
+    return Optional.ofNullable(nodeToValue.get(node));
+  }
 
-    @Override
-    public Optional<String> getEnvironmentVariable(Node node, String name) {
-        // TODO implement
-        return Optional.empty();
-    }
+  @Override
+  public Optional<String> getEnvironmentVariable(Node node, String name) {
+    // TODO implement
+    return Optional.empty();
+  }
 
-    @Override
-    public Optional<Double> getMetric(Node node, String metricName, AttributeFetcher.NodeMetricRegistry registry) {
-        Map<Node, Double> nodeToValue = metricSnitchToNodeToValue.get(AttributeFetcherImpl.getMetricSnitchTag(metricName, registry));
-        if (nodeToValue == null) {
-            return Optional.empty();
-        }
-        return Optional.ofNullable(nodeToValue.get(node));
+  @Override
+  public Optional<Double> getMetric(Node node, String metricName, AttributeFetcher.NodeMetricRegistry registry) {
+    Map<Node, Double> nodeToValue = metricSnitchToNodeToValue.get(AttributeFetcherImpl.getMetricSnitchTag(metricName, registry));
+    if (nodeToValue == null) {
+      return Optional.empty();
     }
+    return Optional.ofNullable(nodeToValue.get(node));
+  }
 
-    @Override
-    public Optional<Double> getMetric(String scope, String metricName) {
-        // TODO implement
-        return Optional.empty();
-    }
+  @Override
+  public Optional<Double> getMetric(String scope, String metricName) {
+    // TODO implement
+    return Optional.empty();
+  }
 }
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 56071da..7f7f89f 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
@@ -25,13 +25,13 @@ import org.apache.solr.cluster.placement.*;
 import java.util.Set;
 
 public class PlacementPlanFactoryImpl implements PlacementPlanFactory {
-    @Override
-    public PlacementPlan createPlacementPlan(PlacementRequest request, Set<ReplicaPlacement> replicaPlacements) {
-        return new PlacementPlanImpl(request, replicaPlacements);
-    }
+  @Override
+  public PlacementPlan createPlacementPlan(PlacementRequest request, Set<ReplicaPlacement> replicaPlacements) {
+    return new PlacementPlanImpl(request, replicaPlacements);
+  }
 
-    @Override
-    public ReplicaPlacement createReplicaPlacement(SolrCollection solrCollection, String shardName, Node node, Replica.ReplicaType replicaType) {
-        return new ReplicaPlacementImpl(solrCollection, shardName, node, replicaType);
-    }
+  @Override
+  public ReplicaPlacement createReplicaPlacement(SolrCollection solrCollection, String shardName, Node node, Replica.ReplicaType replicaType) {
+    return new ReplicaPlacementImpl(solrCollection, shardName, node, replicaType);
+  }
 }
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
index 98d02b7..30cb6ef 100644
--- 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
@@ -85,7 +85,7 @@ public class PlacementPluginConfigImpl implements PlacementPluginConfig {
 
   @Override
   public Long getLongConfig(String configName, long defaultValue) {
-    Long  retval = longConfigs.get(configName);
+    Long retval = longConfigs.get(configName);
     return retval != null ? retval : defaultValue;
   }
 
@@ -181,13 +181,13 @@ public class PlacementPluginConfigImpl implements PlacementPluginConfig {
     PlacementPluginFactory placementPluginFactory;
     try {
       Class<? extends PlacementPluginFactory> factoryClazz =
-              Class.forName(pluginFactoryClassName, true, PlacementPluginConfigImpl.class.getClassLoader())
-                      .asSubclass(PlacementPluginFactory.class);
+          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);
+      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
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
index 555b103..ff3f090 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
@@ -35,8 +35,8 @@ public class PlacementRequestImpl implements PlacementRequest {
   private final EnumMap<Replica.ReplicaType, Integer> countReplicas = new EnumMap<>(Replica.ReplicaType.class);
 
   public PlacementRequestImpl(SolrCollection solrCollection,
-                               Set<String> shardNames, Set<Node> targetNodes,
-                               int countNrtReplicas, int countTlogReplicas, int countPullReplicas) {
+                              Set<String> shardNames, Set<Node> targetNodes,
+                              int countNrtReplicas, int countTlogReplicas, int countPullReplicas) {
     this.solrCollection = solrCollection;
     this.shardNames = shardNames;
     this.targetNodes = targetNodes;
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java
index 47b8f6c..e26a374 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java
@@ -112,9 +112,15 @@ class SimpleClusterAbstractionsImpl {
      * with names equal to existing instances (See {@link ReplicaImpl} constructor).
      */
     public boolean equals(Object obj) {
-      if (obj == null) { return false; }
-      if (obj == this) { return true; }
-      if (obj.getClass() != getClass()) { return false; }
+      if (obj == null) {
+        return false;
+      }
+      if (obj == this) {
+        return true;
+      }
+      if (obj.getClass() != getClass()) {
+        return false;
+      }
       NodeImpl other = (NodeImpl) obj;
       return Objects.equals(this.nodeName, other.nodeName);
     }
@@ -127,7 +133,9 @@ class SimpleClusterAbstractionsImpl {
 
   static class SolrCollectionImpl implements SolrCollection {
     private final String collectionName;
-    /** Map from {@link Shard#getShardName()} to {@link Shard} */
+    /**
+     * Map from {@link Shard#getShardName()} to {@link Shard}
+     */
     private final Map<String, Shard> shards;
     private final DocCollection docCollection;
 
@@ -212,12 +220,18 @@ class SimpleClusterAbstractionsImpl {
 
     private ShardState translateState(Slice.State state) {
       switch (state) {
-        case ACTIVE: return ShardState.ACTIVE;
-        case INACTIVE: return ShardState.INACTIVE;
-        case CONSTRUCTION: return ShardState.CONSTRUCTION;
-        case RECOVERY: return ShardState.RECOVERY;
-        case RECOVERY_FAILED: return ShardState.RECOVERY_FAILED;
-        default: throw new RuntimeException("Unexpected " + state);
+        case ACTIVE:
+          return ShardState.ACTIVE;
+        case INACTIVE:
+          return ShardState.INACTIVE;
+        case CONSTRUCTION:
+          return ShardState.CONSTRUCTION;
+        case RECOVERY:
+          return ShardState.RECOVERY;
+        case RECOVERY_FAILED:
+          return ShardState.RECOVERY_FAILED;
+        default:
+          throw new RuntimeException("Unexpected " + state);
       }
     }
 
@@ -258,15 +272,21 @@ class SimpleClusterAbstractionsImpl {
     }
 
     public boolean equals(Object obj) {
-      if (obj == null) { return false; }
-      if (obj == this) { return true; }
-      if (obj.getClass() != getClass()) { return false; }
+      if (obj == null) {
+        return false;
+      }
+      if (obj == this) {
+        return true;
+      }
+      if (obj.getClass() != getClass()) {
+        return false;
+      }
       ShardImpl other = (ShardImpl) obj;
       return Objects.equals(this.shardName, other.shardName)
-              && Objects.equals(this.collection, other.collection)
-              && Objects.equals(this.shardState, other.shardState)
-              && Objects.equals(this.replicas, other.replicas)
-              && Objects.equals(this.leader, other.leader);
+          && Objects.equals(this.collection, other.collection)
+          && Objects.equals(this.shardState, other.shardState)
+          && Objects.equals(this.replicas, other.replicas)
+          && Objects.equals(this.leader, other.leader);
     }
 
     public int hashCode() {
@@ -316,20 +336,29 @@ class SimpleClusterAbstractionsImpl {
 
     private Replica.ReplicaType translateType(org.apache.solr.common.cloud.Replica.Type type) {
       switch (type) {
-        case NRT: return Replica.ReplicaType.NRT;
-        case TLOG: return Replica.ReplicaType.TLOG;
-        case PULL: return Replica.ReplicaType.PULL;
-        default: throw new RuntimeException("Unexpected " + type);
+        case NRT:
+          return Replica.ReplicaType.NRT;
+        case TLOG:
+          return Replica.ReplicaType.TLOG;
+        case PULL:
+          return Replica.ReplicaType.PULL;
+        default:
+          throw new RuntimeException("Unexpected " + type);
       }
     }
 
     private Replica.ReplicaState translateState(org.apache.solr.common.cloud.Replica.State state) {
       switch (state) {
-        case ACTIVE: return Replica.ReplicaState.ACTIVE;
-        case DOWN: return Replica.ReplicaState.DOWN;
-        case RECOVERING: return Replica.ReplicaState.RECOVERING;
-        case RECOVERY_FAILED: return Replica.ReplicaState.RECOVERY_FAILED;
-        default: throw new RuntimeException("Unexpected " + state);
+        case ACTIVE:
+          return Replica.ReplicaState.ACTIVE;
+        case DOWN:
+          return Replica.ReplicaState.DOWN;
+        case RECOVERING:
+          return Replica.ReplicaState.RECOVERING;
+        case RECOVERY_FAILED:
+          return Replica.ReplicaState.RECOVERY_FAILED;
+        default:
+          throw new RuntimeException("Unexpected " + state);
       }
     }
 
@@ -370,24 +399,34 @@ class SimpleClusterAbstractionsImpl {
      */
     static org.apache.solr.common.cloud.Replica.Type toCloudReplicaType(ReplicaType type) {
       switch (type) {
-        case NRT: return org.apache.solr.common.cloud.Replica.Type.NRT;
-        case TLOG: return org.apache.solr.common.cloud.Replica.Type.TLOG;
-        case PULL: return org.apache.solr.common.cloud.Replica.Type.PULL;
-        default: throw new IllegalArgumentException("Unknown " + type);
+        case NRT:
+          return org.apache.solr.common.cloud.Replica.Type.NRT;
+        case TLOG:
+          return org.apache.solr.common.cloud.Replica.Type.TLOG;
+        case PULL:
+          return org.apache.solr.common.cloud.Replica.Type.PULL;
+        default:
+          throw new IllegalArgumentException("Unknown " + type);
       }
     }
 
     public boolean equals(Object obj) {
-      if (obj == null) { return false; }
-      if (obj == this) { return true; }
-      if (obj.getClass() != getClass()) { return false; }
+      if (obj == null) {
+        return false;
+      }
+      if (obj == this) {
+        return true;
+      }
+      if (obj.getClass() != getClass()) {
+        return false;
+      }
       ReplicaImpl other = (ReplicaImpl) obj;
       return Objects.equals(this.replicaName, other.replicaName)
-              && Objects.equals(this.coreName, other.coreName)
-              && Objects.equals(this.shard, other.shard)
-              && Objects.equals(this.replicaType, other.replicaType)
-              && Objects.equals(this.replicaState, other.replicaState)
-              && Objects.equals(this.node, other.node);
+          && Objects.equals(this.coreName, other.coreName)
+          && Objects.equals(this.shard, other.shard)
+          && Objects.equals(this.replicaType, other.replicaType)
+          && Objects.equals(this.replicaState, other.replicaState)
+          && Objects.equals(this.node, other.node);
     }
 
     public int hashCode() {
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 0dd7340..379e06c 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
@@ -41,13 +41,13 @@ import java.util.stream.Collectors;
  *
  * <pre>
  *
-  curl -X POST -H 'Content-type:application/json' -d '{
-    "set-placement-plugin": {
-      "class": "org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory",
-      "minimalFreeDiskGB": 10,
-      "prioritizedFreeDiskGB": 50
-    }
-  }' http://localhost:8983/api/cluster
+ * curl -X POST -H 'Content-type:application/json' -d '{
+ * "set-placement-plugin": {
+ * "class": "org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory",
+ * "minimalFreeDiskGB": 10,
+ * "prioritizedFreeDiskGB": 50
+ * }
+ * }' http://localhost:8983/api/cluster
  * </pre>
  *
  * <p>The consequence will be the creation of an element in the Zookeeper file {@code /clusterprops.json} as follows:</p>
@@ -65,9 +65,9 @@ import java.util.stream.Collectors;
  *
  * <pre>
  *
-  curl -X POST -H 'Content-type:application/json' -d '{
-    "set-placement-plugin" : null
-  }' http://localhost:8983/api/cluster
+ * curl -X POST -H 'Content-type:application/json' -d '{
+ * "set-placement-plugin" : null
+ * }' http://localhost:8983/api/cluster
  * </pre>
  *
  *
@@ -136,7 +136,9 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
    */
   public static final String REPLICA_TYPE_SYSPROP = "replica_type";
 
-  /** This is the "AZ" name for nodes that do not define an AZ. Should not match a real AZ name (I think we're safe) */
+  /**
+   * This is the "AZ" name for nodes that do not define an AZ. Should not match a real AZ name (I think we're safe)
+   */
   public static final String UNDEFINED_AVAILABILITY_ZONE = "uNd3f1NeD";
 
   /**
@@ -231,7 +233,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
         // failure. Current code does fail if placement is impossible (constraint is at most one replica of a shard on any node).
         for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
           makePlacementDecisions(solrCollection, shardName, availabilityZones, replicaType, request.getCountReplicasToCreate(replicaType),
-                  attrValues, replicaTypeToNodes, coresOnNodes, placementPlanFactory, replicaPlacements);
+              attrValues, replicaTypeToNodes, coresOnNodes, placementPlanFactory, replicaPlacements);
         }
       }
 
@@ -287,7 +289,8 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
      * Also builds the number of existing cores on each node present in the returned EnumMap (2nd member of the returned Pair).
      * Nodes for which the number of cores is not available for whatever reason are excluded from acceptable candidate nodes
      * as it would not be possible to make any meaningful placement decisions.
-     * @param nodes all nodes on which this plugin should compute placement
+     *
+     * @param nodes      all nodes on which this plugin should compute placement
      * @param attrValues attributes fetched for the nodes. This method uses system property {@link #REPLICA_TYPE_SYSPROP} as
      *                   well as the number of cores on each node.
      */
@@ -307,7 +310,8 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
           }
           // We rely later on the fact that the free disk optional is present (see CoresAndDiskComparator), be careful it you change anything here.
           continue;
-        } if (attrValues.getFreeDisk(node).get() < minimalFreeDiskGB) {
+        }
+        if (attrValues.getFreeDisk(node).get() < minimalFreeDiskGB) {
           if (log.isWarnEnabled()) {
             log.warn("Node {} free disk ({}GB) lower than configured minimum {}GB, excluding it from placement decisions.", node.getName(), attrValues.getFreeDisk(node).get(), minimalFreeDiskGB);
           }
@@ -449,7 +453,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
           // This can happen because not enough nodes for the placement request or already too many nodes with replicas of
           // the shard that can't accept new replicas or not enough nodes with enough free disk space.
           throw new PlacementException("Not enough nodes to place " + numReplicas + " replica(s) of type " + replicaType +
-                  " for shard " + shardName + " of collection " + solrCollection.getName());
+              " for shard " + shardName + " of collection " + solrCollection.getName());
         }
 
         AzWithNodes azWithNodes = azWithNodesEntry.getValue();

[lucene] 32/33: Merge branch 'master' into jira/solr-15016

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 345db0e4a44a7d8d230bc5b7efedb62d0d2ca0d5
Merge: b5e5dda 21b8890
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Sat Dec 5 16:35:00 2020 +0100

    Merge branch 'master' into jira/solr-15016

 dev-tools/scripts/LUCENE-3753.patch.hack.pl        | 111 ------------
 dev-tools/scripts/crawl.maven.release.dist.sh      |  68 --------
 dev-tools/scripts/gitignore-gen.sh                 |  21 ---
 dev-tools/scripts/write.stage.maven.build.xml.pl   | 180 --------------------
 lucene/CHANGES.txt                                 |   4 +
 .../org/apache/lucene/search/FieldComparator.java  |  11 ++
 .../apache/lucene/search/LeafFieldComparator.java  |   2 +-
 .../apache/lucene/search/TopFieldCollector.java    |  56 +++---
 .../search/comparators/NumericComparator.java      |  17 +-
 .../search/matchhighlight/MatchHighlighter.java    |   5 +
 .../lucene/search/matchhighlight/Passage.java      |   9 +
 .../intervals/ConjunctionIntervalIterator.java     |   8 +-
 .../lucene/queries/intervals/IntervalIterator.java |   6 +
 .../queries/intervals/OneTimeIntervalSource.java   | 189 +++++++++++++++++++++
 .../queries/intervals/TestIntervalQuery.java       |   6 +
 solr/CHANGES.txt                                   |  48 +++++-
 .../apache/solr/ltr/TestLTRReRankingPipeline.java  |   3 +-
 .../org/apache/solr/ltr/TestLTRScoringQuery.java   |   3 +-
 .../test/org/apache/solr/ltr/TestRerankBase.java   |   3 +-
 .../apache/solr/ltr/norm/TestMinMaxNormalizer.java |   5 +-
 .../solr/ltr/norm/TestStandardNormalizer.java      |   5 +-
 solr/contrib/prometheus-exporter/CHANGES.md        |  20 ---
 .../conf/solr-exporter-config.xml                  |  34 ++++
 .../src/java/org/apache/solr/cloud/Overseer.java   |  16 +-
 .../OverseerCollectionConfigSetProcessor.java      |  11 +-
 .../apache/solr/cloud/OverseerTaskProcessor.java   |  21 ++-
 .../src/java/org/apache/solr/core/NodeConfig.java  |   2 +
 .../src/java/org/apache/solr/core/SolrPaths.java   |  59 -------
 .../org/apache/solr/core/SolrResourceLoader.java   |  20 +--
 .../java/org/apache/solr/core/XmlConfigFile.java   |   5 +-
 .../org/apache/solr/search/SolrCoreParser.java     |  13 +-
 .../apache/solr/servlet/SolrDispatchFilter.java    |  64 ++++++-
 .../OverseerCollectionConfigSetProcessorTest.java  |  18 +-
 .../test/org/apache/solr/cloud/OverseerTest.java   |   5 +-
 .../solr/cloud/TestPullReplicaErrorHandling.java   |   4 +-
 .../org/apache/solr/cloud/ZkControllerTest.java    |  11 +-
 .../org/apache/solr/core/ResourceLoaderTest.java   |   7 +-
 .../org/apache/solr/core/TestCoreContainer.java    |  22 +++
 .../org/apache/solr/core/TestCoreDiscovery.java    |   3 +-
 .../solr/index/WrapperMergePolicyFactoryTest.java  |   2 +-
 .../apache/solr/metrics/SolrMetricManagerTest.java |  94 +++++-----
 .../org/apache/solr/search/TestSolrCoreParser.java |  14 +-
 .../org/apache/solr/search/TestXmlQParser.java     |  18 +-
 .../security/MultiDestinationAuditLoggerTest.java  |   5 +-
 solr/docker/CHANGES.md                             |  28 ---
 solr/solr-ref-guide/src/metrics-reporting.adoc     |   6 +
 solr/solr-ref-guide/src/solr-upgrade-notes.adoc    |  22 ++-
 .../client/solrj/response/QueryResponseTest.java   |  11 +-
 .../solrj/response/TestClusteringResponse.java     |   3 +-
 .../apache/solr/common/util/ContentStreamTest.java |  19 ++-
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   |   2 +
 .../src/java/org/apache/solr/util/TestHarness.java |  17 +-
 .../solr/cloud/MiniSolrCloudClusterTest.java       |  37 ++++
 53 files changed, 706 insertions(+), 667 deletions(-)

diff --cc solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
index ee5caeb,5b18ab0..6127e59
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java
@@@ -122,9 -122,8 +123,10 @@@ public class OverseerCollectionConfigSe
    private static CoreContainer coreContainerMock;
    private static UpdateShardHandler updateShardHandlerMock;
    private static HttpClient httpClientMock;
 +  @SuppressWarnings("rawtypes")
 +  private static PlacementPluginFactory placementPluginFactoryMock;
-   
+   private static SolrMetricsContext solrMetricsContextMock;
 -  
++
    private static ObjectCache objectCache;
    private Map<String, byte[]> zkClientData = new HashMap<>();
    private final Map<String, ClusterState.CollectionRef> collectionsSet = new HashMap<>();
@@@ -183,7 -183,7 +186,8 @@@
      coreContainerMock = mock(CoreContainer.class);
      updateShardHandlerMock = mock(UpdateShardHandler.class);
      httpClientMock = mock(HttpClient.class);
 +    placementPluginFactoryMock = mock(PlacementPluginFactory.class);
+     solrMetricsContextMock = mock(SolrMetricsContext.class);
    }
    
    @AfterClass
@@@ -208,7 -208,7 +212,8 @@@
      coreContainerMock = null;
      updateShardHandlerMock = null;
      httpClientMock = null;
 +    placementPluginFactoryMock = null;
+     solrMetricsContextMock = null;
    }
    
    @Before
@@@ -238,7 -238,7 +243,8 @@@
      reset(coreContainerMock);
      reset(updateShardHandlerMock);
      reset(httpClientMock);
 +    reset(placementPluginFactoryMock);
+     reset(solrMetricsContextMock);
  
      zkClientData.clear();
      collectionsSet.clear();

[lucene] 06/33: SOLR-15004: Add a scalability test.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 1d50edbc46000c9374ae13c92a8fbe07120178de
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Fri Nov 20 14:49:30 2020 +0100

    SOLR-15004: Add a scalability test.
---
 .../impl/AffinityPlacementFactoryTest.java         | 91 ++++++++++++++++++++++
 1 file changed, 91 insertions(+)

diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
index 35bfcfc..48c7eb6 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
@@ -19,6 +19,7 @@ package org.apache.solr.cluster.placement.impl;
 
 import org.apache.solr.cluster.Cluster;
 import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
 import org.apache.solr.cluster.Shard;
 import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.*;
@@ -26,14 +27,22 @@ import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * Unit test for {@link AffinityPlacementFactory}
  */
 public class AffinityPlacementFactoryTest extends Assert {
+    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
     private static PlacementPlugin plugin;
 
@@ -100,4 +109,86 @@ public class AffinityPlacementFactoryTest extends Assert {
         ReplicaPlacement rp = pp.getReplicaPlacements().iterator().next();
         assertEquals(hasExistingCollection ? node2 : node1, rp.getNode());
     }
+
+    @Test
+    public void testScalability() throws Exception {
+        log.info("==== numNodes ====");
+        runTestScalability(1000, 100, 40, 40, 20);
+        runTestScalability(2000, 100, 40, 40, 20);
+        runTestScalability(5000, 100, 40, 40, 20);
+        runTestScalability(10000, 100, 40, 40, 20);
+        runTestScalability(20000, 100, 40, 40, 20);
+        log.info("==== numShards ====");
+        runTestScalability(5000, 100, 40, 40, 20);
+        runTestScalability(5000, 200, 40, 40, 20);
+        runTestScalability(5000, 500, 40, 40, 20);
+        runTestScalability(5000, 1000, 40, 40, 20);
+        runTestScalability(5000, 2000, 40, 40, 20);
+        log.info("==== numReplicas ====");
+        runTestScalability(5000, 100, 100, 0, 0);
+        runTestScalability(5000, 100, 200, 0, 0);
+        runTestScalability(5000, 100, 500, 0, 0);
+        runTestScalability(5000, 100, 1000, 0, 0);
+        runTestScalability(5000, 100, 2000, 0, 0);
+    }
+
+    private void runTestScalability(int numNodes, int numShards,
+                                    int nrtReplicas, int tlogReplicas,
+                                    int pullReplicas) throws Exception {
+
+        int REPLICAS_PER_SHARD = nrtReplicas + tlogReplicas + pullReplicas;
+        int TOTAL_REPLICAS = numShards * REPLICAS_PER_SHARD;
+
+        String collectionName = "testCollection";
+
+        final Set<Node> liveNodes = new HashSet<>();
+        final Map<Node, Long> nodeToFreeDisk = new HashMap<>();
+        final Map<Node, Integer> nodeToCoreCount = new HashMap<>();
+        for (int i = 0; i < numNodes; i++) {
+            Node node = new ClusterAbstractionsForTest.NodeImpl("node_" + i);
+            liveNodes.add(node);
+            nodeToFreeDisk.put(node, Long.valueOf(numNodes));
+            nodeToCoreCount.put(node, 0);
+        }
+        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(collectionName, Map.of());
+        Map<String, Shard> shards = PluginTestHelper.createShardsAndReplicas(solrCollection, numShards, 0, Set.of());
+        solrCollection.setShards(shards);
+
+        Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, Map.of());
+        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, shards.keySet(), liveNodes,
+            nrtReplicas, tlogReplicas, pullReplicas);
+
+        AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
+        AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
+        PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+
+        long start = System.nanoTime();
+        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+        long end = System.nanoTime();
+        log.info("ComputePlacement: {} nodes, {} shards, {} total replicas, elapsed time {} ms.", numNodes, numShards, TOTAL_REPLICAS, TimeUnit.NANOSECONDS.toMillis(end - start));
+        assertEquals("incorrect number of calculated placements", TOTAL_REPLICAS,
+            pp.getReplicaPlacements().size());
+        // check that replicas are correctly placed
+        Map<Node, AtomicInteger> replicasPerNode = new HashMap<>();
+        Map<Node, Set<String>> shardsPerNode = new HashMap<>();
+        Map<String, AtomicInteger> replicasPerShard = new HashMap<>();
+        Map<Replica.ReplicaType, AtomicInteger> replicasByType = new HashMap<>();
+        for (ReplicaPlacement placement : pp.getReplicaPlacements()) {
+            replicasPerNode.computeIfAbsent(placement.getNode(), n -> new AtomicInteger()).incrementAndGet();
+            shardsPerNode.computeIfAbsent(placement.getNode(), n -> new HashSet<>()).add(placement.getShardName());
+            replicasByType.computeIfAbsent(placement.getReplicaType(), t -> new AtomicInteger()).incrementAndGet();
+            replicasPerShard.computeIfAbsent(placement.getShardName(), s -> new AtomicInteger()).incrementAndGet();
+        }
+        int perNode = TOTAL_REPLICAS > numNodes ? TOTAL_REPLICAS / numNodes : 1;
+        replicasPerNode.forEach((node, count) -> {
+            assertEquals(count.get(), perNode);
+        });
+        shardsPerNode.forEach((node, names) -> {
+            assertEquals(names.size(), perNode);
+        });
+
+        replicasPerShard.forEach((shard, count) -> {
+            assertEquals(count.get(), REPLICAS_PER_SHARD);
+        });
+    }
 }

[lucene] 04/33: SOLR-15004: Add integration test. Move some properties so that they are accessible (and can be validated) in ClusterProperties. Fix a bug in update().

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit a6a47a094126c5e06ed656a6603e821cb4c7bf63
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Nov 19 15:28:31 2020 +0100

    SOLR-15004: Add integration test. Move some properties so that they are accessible
    (and can be validated) in ClusterProperties. Fix a bug in update().
---
 .../cluster/placement/PlacementPluginConfig.java   |  8 ++
 .../placement/impl/PlacementPluginConfigImpl.java  | 20 ++---
 .../java/org/apache/solr/handler/ClusterAPI.java   |  8 +-
 .../solr/handler/admin/ContainerPluginsApi.java    |  2 +-
 .../impl/PlacementPluginIntegrationTest.java       | 94 ++++++++++++++++++++++
 .../solr/common/cloud/ClusterProperties.java       |  8 +-
 .../apache/solr/common/cloud/ZkStateReader.java    |  9 ++-
 7 files changed, 129 insertions(+), 20 deletions(-)

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 a39390f..4f17051 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
@@ -68,6 +68,14 @@ package org.apache.solr.cluster.placement;
  * </pre>
  */
 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/impl/PlacementPluginConfigImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginConfigImpl.java
index e6130a3..b21cbfd 100644
--- 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
@@ -38,12 +38,6 @@ import org.apache.solr.common.util.Utils;
  * {@link org.apache.solr.cloud.api.collections.Assign} class.</p>
  */
 public class PlacementPluginConfigImpl implements PlacementPluginConfig {
-  /**
-   * The key in {@code clusterprops.json} under which the plugin factory and the plugin configuration are defined.
-   */
-  final public static String PLACEMENT_PLUGIN_CONFIG_KEY = "placement-plugin";
-  /** Name of the property containing the factory class */
-  final public static String CONFIG_CLASS = "class";
 
   // Separating configs into typed maps based on the element names in solr.xml
   private final Map<String, String> stringConfigs;
@@ -126,18 +120,18 @@ public class PlacementPluginConfigImpl implements PlacementPluginConfig {
 
     for (Map.Entry<String, Object> e : pluginConfig.entrySet()) {
       String key = e.getKey();
-      if (CONFIG_CLASS.equals(key)) {
+      if (PlacementPluginConfig.FACTORY_CLASS.equals(key)) {
         continue;
       }
 
       if (key == null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Missing config name attribute in parameter of " + PLACEMENT_PLUGIN_CONFIG_KEY);
+        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 " + PLACEMENT_PLUGIN_CONFIG_KEY);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Missing config value for parameter " + key + " of " + PlacementPluginConfig.PLACEMENT_PLUGIN_CONFIG_KEY);
       }
 
       if (value instanceof String) {
@@ -150,7 +144,7 @@ public class PlacementPluginConfigImpl implements PlacementPluginConfig {
         doubleConfigs.put(key, (Double) value);
       } else {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unsupported config type " + value.getClass().getName() +
-            " for parameter " + key + " of " + PLACEMENT_PLUGIN_CONFIG_KEY);
+            " for parameter " + key + " of " + PlacementPluginConfig.PLACEMENT_PLUGIN_CONFIG_KEY);
       }
     }
 
@@ -172,13 +166,13 @@ public class PlacementPluginConfigImpl implements PlacementPluginConfig {
   @SuppressWarnings({"unchecked"})
   public static PlacementPlugin getPlacementPlugin(SolrCloudManager solrCloudManager) {
     Map<String, Object> props = solrCloudManager.getClusterStateProvider().getClusterProperties();
-    Map<String, Object> pluginConfigMap = (Map<String, Object>) props.get(PLACEMENT_PLUGIN_CONFIG_KEY);
+    Map<String, Object> pluginConfigMap = (Map<String, Object>) props.get(PlacementPluginConfig.PLACEMENT_PLUGIN_CONFIG_KEY);
 
     if (pluginConfigMap == null) {
       return null;
     }
 
-    String pluginFactoryClassName = (String) pluginConfigMap.get(CONFIG_CLASS);
+    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
@@ -193,7 +187,7 @@ public class PlacementPluginConfigImpl implements PlacementPluginConfig {
       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 " + PLACEMENT_PLUGIN_CONFIG_KEY, e);
+              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
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 f3f2816..605dbb6 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,7 @@ 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.impl.PlacementPluginConfigImpl;
+import org.apache.solr.cluster.placement.PlacementPluginConfig;
 import org.apache.solr.common.MapWriterMap;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.annotation.JsonProperty;
@@ -250,14 +250,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(PlacementPluginConfigImpl.CONFIG_CLASS)) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Must contain " + PlacementPluginConfigImpl.CONFIG_CLASS + " attribute (or be null)");
+      if (!(placementPluginConfig == null) && !placementPluginConfig.containsKey(PlacementPluginConfig.FACTORY_CLASS)) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Must contain " + PlacementPluginConfig.FACTORY_CLASS + " attribute (or be null)");
       }
       try {
         clusterProperties.update(placementPluginConfig == null?
             null:
             new MapWriterMap(placementPluginConfig),
-            PlacementPluginConfigImpl.PLACEMENT_PLUGIN_CONFIG_KEY);
+            PlacementPluginConfig.PLACEMENT_PLUGIN_CONFIG_KEY);
       } catch (Exception e) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error in API", e);
       }
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 57bcbef..3b013ca 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
@@ -50,7 +50,7 @@ import static org.apache.lucene.util.IOUtils.closeWhileHandlingException;
 public class ContainerPluginsApi {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static final String PLUGIN = "plugin";
+  public static final String PLUGIN = ZkStateReader.CONTAINER_PLUGINS;
   private final Supplier<SolrZkClient> zkClientSupplier;
   private final CoreContainer coreContainer;
   public final Read readAPI = new Read();
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
new file mode 100644
index 0000000..54cd081
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
@@ -0,0 +1,94 @@
+package org.apache.solr.cluster.placement.impl;
+
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+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.plugins.SamplePluginMinimizeCores;
+import org.apache.solr.common.cloud.ClusterProperties;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static java.util.Collections.singletonMap;
+
+/**
+ *
+ */
+public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
+
+  private static final String COLLECTION = PlacementPluginIntegrationTest.class.getName() + "_collection";
+
+  private static ClusterProperties clusterProperties;
+  private static SolrCloudManager cloudManager;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    // placement plugins need metrics
+    System.setProperty("metricsEnabled", "true");
+    configureCluster(3)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
+    clusterProperties = new ClusterProperties(cluster.getZkClient());
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    cluster.deleteAllCollections();
+    V2Request req = new V2Request.Builder("/cluster")
+        .forceV2(true)
+        .POST()
+        .withPayload(singletonMap("set-placement-plugin", Map.of()))
+        .build();
+    req.process(cluster.getSolrClient());
+
+  }
+
+  @Test
+  public void testMinimizeCores() throws Exception {
+    Map<String, Object> config = Map.of(PlacementPluginConfig.FACTORY_CLASS, SamplePluginMinimizeCores.Factory.class.getName());
+    V2Request req = new V2Request.Builder("/cluster")
+        .forceV2(true)
+        .POST()
+        .withPayload(singletonMap("set-placement-plugin", config))
+        .build();
+    req.process(cluster.getSolrClient());
+
+    CollectionAdminResponse rsp = CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 2)
+        .process(cluster.getSolrClient());
+    assertTrue(rsp.isSuccess());
+    cluster.waitForActiveCollection(COLLECTION, 2, 4);
+    // use Solr-specific API to verify the expected placements
+    ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
+    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();
+    });
+    int maxCores = 0;
+    int minCores = Integer.MAX_VALUE;
+    for (Map.Entry<String, AtomicInteger> entry : coresByNode.entrySet()) {
+      assertTrue("too few cores on node " + entry.getKey() + ": " + entry.getValue(),
+          entry.getValue().get() > 0);
+      if (entry.getValue().get() > maxCores) {
+        maxCores = entry.getValue().get();
+      }
+      if (entry.getValue().get() < minCores) {
+        minCores = entry.getValue().get();
+      }
+    }
+    assertEquals("max cores too high", 2, maxCores);
+    assertEquals("min cores too low", 1, minCores);
+  }
+
+}
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java
index 0460c46..4e91d47 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java
@@ -127,7 +127,13 @@ public class ClusterProperties {
   @SuppressWarnings("unchecked")
   public void update(MapWriter obj, String... path) throws KeeperException, InterruptedException{
     client.atomicUpdate(ZkStateReader.CLUSTER_PROPS, bytes -> {
-      Map<String, Object> zkJson = (Map<String, Object>) Utils.fromJSON(bytes);
+      Map<String, Object> zkJson;
+      if (bytes == null) {
+        // no previous properties - initialize
+        zkJson = new LinkedHashMap<>();
+      } else {
+        zkJson = (Map<String, Object>) Utils.fromJSON(bytes);
+      }
       Utils.setObjectByPath(zkJson, Arrays.asList(path), obj);
       return Utils.toJSON(zkJson);
     });
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 4273163..8a61e56 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -138,6 +138,10 @@ public class ZkStateReader implements SolrCloseable {
 
   public static final String REPLICA_TYPE = "type";
 
+  public static final String CONTAINER_PLUGINS = "plugin";
+
+  public static final String PLACEMENT_PLUGIN = "placement-plugin";
+
   /**
    * A view of the current state of all collections.
    */
@@ -222,7 +226,10 @@ public class ZkStateReader implements SolrCloseable {
       MAX_CORES_PER_NODE,
       SAMPLE_PERCENTAGE,
       SOLR_ENVIRONMENT,
-      CollectionAdminParams.DEFAULTS);
+      CollectionAdminParams.DEFAULTS,
+      CONTAINER_PLUGINS,
+      PLACEMENT_PLUGIN
+      );
 
   /**
    * Returns config set name for collection.

[lucene] 24/33: Merge branch 'master' into jira/solr-15016

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 0cdd738b25fcf1efbbf42e1c81d1bebff5363946
Merge: 6853640 d99c166
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Wed Dec 2 13:29:10 2020 +0100

    Merge branch 'master' into jira/solr-15016

 dev-tools/scripts/addVersion.py                    |   1 +
 gradle/solr/packaging.gradle                       |   3 +-
 lucene/CHANGES.txt                                 |  17 +
 .../lucene/analysis/ja/JapaneseTokenizer.java      |  59 +--
 .../lucene/analysis/ja/TestJapaneseAnalyzer.java   |  14 +-
 .../lucene/analysis/ja/TestJapaneseTokenizer.java  |   6 +
 .../apache/lucene/analysis/ja/TestSearchMode.java  |   5 -
 .../apache/lucene/codecs/lucene84/PForUtil.java    |   2 +-
 .../org/apache/lucene/document/LatLonShape.java    |   3 +-
 .../document/LatLonShapeBoundingBoxQuery.java      |   3 +-
 .../java/org/apache/lucene/document/XYShape.java   |   3 +-
 .../src/java/org/apache/lucene/geo/Circle2D.java   |   2 +-
 .../src/java/org/apache/lucene/geo/Line2D.java     |   6 +-
 .../src/java/org/apache/lucene/geo/Polygon2D.java  |   2 +-
 .../java/org/apache/lucene/geo/Rectangle2D.java    |   4 +-
 .../org/apache/lucene/index/DocumentsWriter.java   |  20 +-
 .../lucene/index/DocumentsWriterStallControl.java  |   4 +
 .../org/apache/lucene/util/hnsw/HnswGraph.java     |   2 +-
 .../apache/lucene/document/TestLatLonShape.java    |  74 ++++
 .../org/apache/lucene/document/TestXYShape.java    |  73 ++++
 .../org/apache/lucene/index/TestIndexWriter.java   |  43 ++
 .../org/apache/lucene/util/TestVectorUtil.java     |   8 +-
 ...orTest.java => TestAllGroupHeadsCollector.java} |   2 +-
 ...lectorTest.java => TestAllGroupsCollector.java} |   2 +-
 ...ockGroupingTest.java => TestBlockGrouping.java} |   2 +-
 ...rTest.java => TestDistinctValuesCollector.java} |   2 +-
 ...Test.java => TestDoubleRangeGroupSelector.java} |   2 +-
 ...ectorTest.java => TestGroupFacetCollector.java} |   2 +-
 ...pingSearchTest.java => TestGroupingSearch.java} |   2 +-
 ...orTest.java => TestLongRangeGroupSelector.java} |   2 +-
 ...electorTest.java => TestTermGroupSelector.java} |   2 +-
 .../{TopGroupsTest.java => TestTopGroups.java}     |   2 +-
 ...Test.java => TestValueSourceGroupSelector.java} |   2 +-
 ...rmsQueryTest.java => TestCommonTermsQuery.java} |   6 +-
 ... => TestIndexAndTaxonomyReplicationClient.java} |   2 +-
 ...Test.java => TestIndexAndTaxonomyRevision.java} |   2 +-
 ...ntTest.java => TestIndexReplicationClient.java} |   2 +-
 ...dexRevisionTest.java => TestIndexRevision.java} |   2 +-
 ...eplicatorTest.java => TestLocalReplicator.java} |   2 +-
 ...SessionTokenTest.java => TestSessionToken.java} |   2 +-
 ...ReplicatorTest.java => TestHttpReplicator.java} |   2 +-
 ...sQueryTest.java => TestFuzzyLikeThisQuery.java} |   2 +-
 ...StrategyTest.java => TestDistanceStrategy.java} |   4 +-
 .../{PortedSolr3Test.java => TestPortedSolr3.java} |   4 +-
 .../{SpatialArgsTest.java => TestSpatialArgs.java} |   2 +-
 ...trategyTest.java => TestCompositeStrategy.java} |   2 +-
 ...NRStrategyTest.java => TestDateNRStrategy.java} |   2 +-
 ...unterTest.java => TestHeatmapFacetCounter.java} |   2 +-
 .../{JtsPolygonTest.java => TestJtsPolygon.java}   |   4 +-
 ...eFacetsTest.java => TestNumberRangeFacets.java} |   2 +-
 ...ava => TestRandomSpatialOpFuzzyPrefixTree.java} |   4 +-
 ...a => TestRandomSpatialOpFuzzyPrefixTree50.java} |   2 +-
 ...xTreeTest.java => TestDateRangePrefixTree.java} |   4 +-
 ...S2PrefixTreeTest.java => TestS2PrefixTree.java} |   2 +-
 ...fixTreeTest.java => TestSpatialPrefixTree.java} |   2 +-
 ...sParserTest.java => TestSpatialArgsParser.java} |   2 +-
 ...rategyTest.java => TestSerializedStrategy.java} |   2 +-
 solr/CHANGES.txt                                   |  26 +-
 .../src/java/org/apache/solr/ltr/LTRRescorer.java  | 153 ++++---
 .../java/org/apache/solr/ltr/LTRScoringQuery.java  |   4 +
 .../solr/ltr/SolrQueryRequestContextUtils.java     |  12 +-
 .../apache/solr/ltr/interleaving/Interleaving.java |  46 +++
 .../solr/ltr/interleaving/InterleavingResult.java  |  35 +-
 .../ltr/interleaving/LTRInterleavingQuery.java     |  75 ++++
 .../ltr/interleaving/LTRInterleavingRescorer.java  | 162 ++++++++
 .../interleaving/LTRInterleavingScoringQuery.java  |  53 +++
 .../OriginalRankingLTRScoringQuery.java}           |  20 +-
 .../algorithms/TeamDraftInterleaving.java          | 127 ++++++
 .../ltr/interleaving/algorithms}/package-info.java |   4 +-
 .../solr/ltr/interleaving}/package-info.java       |   4 +-
 .../LTRFeatureLoggerTransformerFactory.java        | 187 ++++++---
 .../LTRInterleavingTransformerFactory.java         | 114 ++++++
 .../apache/solr/ltr/search/LTRQParserPlugin.java   | 155 ++++----
 .../java/org/apache/solr/ltr/search/LTRQuery.java  |  80 ++++
 solr/contrib/ltr/src/java/overview.html            |   2 +-
 .../solr/collection1/conf/solrconfig-ltr.xml       |   9 +
 .../org/apache/solr/ltr/TestLTRQParserExplain.java | 160 ++++++++
 .../org/apache/solr/ltr/TestLTRQParserPlugin.java  |  16 +-
 .../test/org/apache/solr/ltr/TestLTRWithSort.java  | 103 +++++
 .../interleaving/TestLTRQParserInterleaving.java   | 206 ++++++++++
 .../algorithms/TeamDraftInterleavingTest.java      | 170 ++++++++
 .../transform/TestFeatureLoggerTransformer.java    | 400 +++++++++++++++++++
 .../transform/TestInterleavingTransformer.java     | 277 +++++++++++++
 solr/contrib/prometheus-exporter/CHANGES.md        |  20 +
 solr/contrib/prometheus-exporter/bin/solr-exporter |  15 -
 .../prometheus-exporter/bin/solr-exporter.cmd      |   5 +-
 solr/contrib/prometheus-exporter/build.gradle      |  51 ++-
 solr/contrib/prometheus-exporter/conf/log4j2.xml   |  40 ++
 .../prometheus/exporter/MetricsConfiguration.java  |  72 +++-
 .../solr/prometheus/exporter/SolrExporter.java     |  15 +-
 .../solr/prometheus/scraper/SolrCloudScraper.java  |  16 +-
 .../org/apache/solr/prometheus/utils/Helpers.java  |  13 +-
 .../src/java/org/apache/solr/cloud/CloudUtil.java  |   3 +-
 .../org/apache/solr/cloud/RecoveryStrategy.java    |  13 +-
 .../solr/cloud/ShardLeaderElectionContextBase.java |   4 +-
 .../java/org/apache/solr/cloud/ZkController.java   |  23 +-
 .../solr/cloud/api/collections/AddReplicaCmd.java  |   1 -
 .../cloud/api/collections/CreateCollectionCmd.java |   1 -
 .../OverseerCollectionMessageHandler.java          |  14 +-
 .../solr/cloud/api/collections/SplitShardCmd.java  |   1 -
 .../apache/solr/cloud/overseer/SliceMutator.java   |  18 +-
 .../plugins/AffinityPlacementFactory.java          | 166 +++++---
 .../cluster/placement/plugins/package-info.java    |   2 +-
 .../java/org/apache/solr/core/BlobRepository.java  |   3 +-
 .../apache/solr/core/CachingDirectoryFactory.java  |  15 +-
 .../src/java/org/apache/solr/core/CloudConfig.java |   2 +-
 .../src/java/org/apache/solr/core/NodeConfig.java  |  36 +-
 .../src/java/org/apache/solr/core/SolrConfig.java  |  68 ++--
 .../java/org/apache/solr/core/SolrXmlConfig.java   |  49 ++-
 .../apache/solr/core/StandardDirectoryFactory.java |  15 +-
 .../src/java/org/apache/solr/core/ZkContainer.java |  19 +-
 .../apache/solr/filestore/DistribPackageStore.java |   2 +-
 .../solr/handler/ContentStreamHandlerBase.java     |  65 +--
 .../apache/solr/handler/admin/ClusterStatus.java   |  26 +-
 .../solr/handler/admin/MetricsHistoryHandler.java  |   3 +-
 .../solr/handler/admin/RebalanceLeaders.java       |   2 +-
 .../solr/handler/admin/RequestSyncShardOp.java     |   1 -
 .../solr/handler/admin/ZookeeperInfoHandler.java   |  10 +-
 .../java/org/apache/solr/servlet/HttpSolrCall.java |  82 ++--
 .../apache/solr/servlet/SolrDispatchFilter.java    | 103 ++---
 .../src/java/org/apache/solr/util/SolrCLI.java     |   2 +
 .../test/org/apache/solr/TestRandomDVFaceting.java |   1 +
 .../solr/cloud/ClusterStateMockUtilTest.java       |   1 +
 .../org/apache/solr/cloud/DeleteReplicaTest.java   |   6 +-
 .../org/apache/solr/cloud/LeaderElectionTest.java  |  32 +-
 .../org/apache/solr/cloud/NodeMutatorTest.java     |   2 +
 .../test/org/apache/solr/cloud/OverseerTest.java   |  69 ++--
 .../cloud/TestLeaderElectionWithEmptyReplica.java  |   4 +-
 .../solr/cloud/TestMiniSolrCloudClusterSSL.java    |   3 +
 .../solr/cloud/TestRandomRequestDistribution.java  |   5 +-
 .../CollectionsAPIDistributedZkTest.java           |   5 +-
 .../solr/cloud/api/collections/ShardSplitTest.java |   5 +-
 .../apache/solr/cluster/placement/Builders.java    | 140 ++++++-
 .../placement/ClusterAbstractionsForTest.java      |  27 --
 .../impl/PlacementPluginIntegrationTest.java       |  21 +-
 .../plugins/AffinityPlacementFactoryTest.java      | 440 +++++++++++++++++++--
 .../org/apache/solr/core/TestMinimalConfig.java    |  45 +++
 .../core/snapshots/TestSolrCloudSnapshots.java     |  13 +-
 .../solr/core/snapshots/TestSolrCoreSnapshots.java |   6 +-
 .../org/apache/solr/handler/TestBlobHandler.java   |   3 +-
 .../org/apache/solr/handler/TestConfigReload.java  |   2 +-
 .../solr/handler/TestHdfsBackupRestoreCore.java    |   7 +-
 .../org/apache/solr/handler/TestReqParamsAPI.java  |   2 +-
 .../solr/handler/TestSolrConfigHandlerCloud.java   |   4 +-
 .../handler/TestSolrConfigHandlerConcurrent.java   |   2 +-
 .../handler/component/CloudReplicaSourceTest.java  |   1 +
 .../apache/solr/index/hdfs/CheckHdfsIndexTest.java |   2 +
 .../solr/uninverting/TestFieldCacheSort.java       |   3 +-
 .../apache/solr/update/SolrCmdDistributorTest.java |  25 +-
 solr/docker/CHANGES.md                             |  28 ++
 solr/docker/Dockerfile                             |   6 +-
 solr/docker/include/scripts/docker-entrypoint.sh   |   3 +
 solr/docker/include/scripts/init-var-solr          |  16 +-
 solr/docker/include/scripts/solr-create            |   3 -
 solr/docker/include/scripts/solr-foreground        |   3 -
 solr/docker/include/scripts/solr-precreate         |   3 -
 .../cases/empty-varsolr-vol-solr-nocopy/test.sh    |   1 +
 .../conf/solrconfig.xml                            |   1 -
 .../src/common-query-parameters.adoc               |   1 -
 solr/solr-ref-guide/src/config-api.adoc            |   3 +-
 solr/solr-ref-guide/src/css/navs.css               |   2 +-
 solr/solr-ref-guide/src/distributed-requests.adoc  |   4 -
 solr/solr-ref-guide/src/enabling-ssl.adoc          |   2 +
 .../velocity-search-ui/techproducts_browse.png     | Bin 74854 -> 0 bytes
 solr/solr-ref-guide/src/learning-to-rank.adoc      | 104 ++++-
 solr/solr-ref-guide/src/metrics-reporting.adoc     |  82 ++--
 .../client/solrj/impl/BaseCloudSolrClient.java     |   8 +-
 .../solr/client/solrj/impl/Http2SolrClient.java    |   8 +
 .../RequestReplicaListTransformerGenerator.java    |  15 +-
 .../apache/solr/common/cloud/ClusterStateUtil.java |   2 +-
 .../java/org/apache/solr/common/cloud/Replica.java |  22 +-
 .../org/apache/solr/common/cloud/UrlScheme.java    |  73 ++++
 .../apache/solr/common/cloud/ZkCoreNodeProps.java  |  27 +-
 .../solr/common/cloud/ZkMaintenanceUtils.java      |   2 +-
 .../org/apache/solr/common/cloud/ZkNodeProps.java  |  54 ++-
 .../apache/solr/common/cloud/ZkStateReader.java    |  16 +-
 .../apache/solr/common/params/CommonParams.java    |   7 -
 .../solrj/impl/CloudHttp2SolrClientTest.java       |  61 +--
 .../client/solrj/impl/CloudSolrClientTest.java     |  61 +--
 .../client/solrj/impl/Http2SolrClientTest.java     |   6 +
 .../routing/NodePreferenceRulesComparatorTest.java |  45 ++-
 ...RequestReplicaListTransformerGeneratorTest.java |  64 +--
 .../ShufflingReplicaListTransformerTest.java       |   4 +-
 .../solr/common/cloud/TestZkMaintenanceUtils.java  |  20 +
 .../apache/solr/common/cloud/UrlSchemeTest.java    |  56 +++
 .../solr/common/params/CommonParamsTest.java       |   2 -
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   |  10 +-
 .../solr/cloud/AbstractFullDistribZkTestBase.java  |  22 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |  26 +-
 .../src/java/org/apache/solr/util/TestHarness.java |  15 +-
 .../solr/cloud/MiniSolrCloudClusterTest.java       |  74 ++++
 191 files changed, 4666 insertions(+), 1206 deletions(-)

diff --cc solr/core/src/test/org/apache/solr/cluster/placement/impl/PlacementPluginIntegrationTest.java
index 8c218e7,ac325b1..e3c9a27
--- 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
@@@ -3,20 -20,14 +20,22 @@@ package org.apache.solr.cluster.placeme
  import org.apache.solr.client.solrj.cloud.SolrCloudManager;
  import org.apache.solr.client.solrj.request.CollectionAdminRequest;
  import org.apache.solr.client.solrj.request.V2Request;
 +import org.apache.solr.client.solrj.request.beans.PluginMeta;
  import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 -import org.apache.solr.cloud.MiniSolrCloudCluster;
 +import org.apache.solr.client.solrj.response.V2Response;
  import org.apache.solr.cloud.SolrCloudTestCase;
 -import org.apache.solr.cluster.placement.PlacementPluginConfig;
 +import org.apache.solr.cluster.placement.PlacementPluginFactory;
 +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.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.TimeOut;
++
  import org.junit.After;
  import org.junit.BeforeClass;
  import org.junit.Test;
diff --cc solr/core/src/test/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactoryTest.java
index 61907be,7e240b6..9cbdc3d
--- 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,9 +26,9 @@@ import org.apache.solr.cluster.SolrColl
  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.apache.solr.cluster.placement.impl.PlacementRequestImpl;
+ import org.apache.solr.common.util.Pair;
  import org.junit.BeforeClass;
  import org.junit.Test;
  import org.slf4j.Logger;
@@@ -47,13 -49,14 +49,15 @@@ public class AffinityPlacementFactoryTe
  
    private static PlacementPlugin plugin;
  
+   private final static long MINIMAL_FREE_DISK_GB = 10L;
+   private final static long PRIORITIZED_FREE_DISK_GB = 50L;
+ 
    @BeforeClass
    public static void setupPlugin() {
-     AffinityPlacementConfig config = new AffinityPlacementConfig(10L, 50L);
 -    PlacementPluginConfig config = PlacementPluginConfigImpl.createConfigFromProperties(
 -        Map.of("minimalFreeDiskGB", MINIMAL_FREE_DISK_GB, "prioritizedFreeDiskGB", PRIORITIZED_FREE_DISK_GB));
 -    plugin = new AffinityPlacementFactory().createPluginInstance(config);
++    AffinityPlacementConfig config = new AffinityPlacementConfig(MINIMAL_FREE_DISK_GB, PRIORITIZED_FREE_DISK_GB);
 +    AffinityPlacementFactory factory = new AffinityPlacementFactory();
 +    factory.configure(config);
 +    plugin = factory.createPluginInstance();
-     ((AffinityPlacementFactory.AffinityPlacementPlugin) plugin).setRandom(random());
    }
  
    @Test

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

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/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;
   }

[lucene] 18/33: SOLR-15004: Some renaming to clarify the purpose of plugin config params.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 5705765b37b03a5c23f5384f6171d1b82333dbcd
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Wed Nov 25 20:25:16 2020 +0100

    SOLR-15004: Some renaming to clarify the purpose of plugin config params.
---
 .../plugins/AffinityPlacementFactory.java          | 48 ++++++++++++----------
 .../plugins/AffinityPlacementFactoryTest.java      | 46 ++++++++++++++++++---
 2 files changed, 66 insertions(+), 28 deletions(-)

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 ddddfd7..9690a86 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
@@ -140,6 +140,20 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
   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";
+
+  /**
    * 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
    * having to call a constructor with more parameters (if we were to instantiate the plugin class directly without going
@@ -150,9 +164,9 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
 
   @Override
   public PlacementPlugin createPluginInstance(PlacementPluginConfig config) {
-    final long minimalFreeDiskGB = config.getLongConfig("minimalFreeDiskGB", 20L);
-    final long deprioritizedFreeDiskGB = config.getLongConfig("deprioritizedFreeDiskGB", 100L);
-    return new AffinityPlacementPlugin(minimalFreeDiskGB, deprioritizedFreeDiskGB);
+    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);
   }
 
   /**
@@ -161,28 +175,18 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
    */
   static class AffinityPlacementPlugin implements PlacementPlugin {
 
-    /**
-     * 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.
-     */
     private final 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).
-     */
-    private final long deprioritizedFreeDiskGB;
+    private final long prioritizedFreeDiskGB;
 
     private Random random = new Random();
 
     /**
      * The factory has decoded the configuration for the plugin instance and passes it the parameters it needs.
      */
-    private AffinityPlacementPlugin(long minimalFreeDiskGB, long deprioritizedFreeDiskGB) {
+    private AffinityPlacementPlugin(long minimalFreeDiskGB, long prioritizedFreeDiskGB) {
       this.minimalFreeDiskGB = minimalFreeDiskGB;
-      this.deprioritizedFreeDiskGB = deprioritizedFreeDiskGB;
+      this.prioritizedFreeDiskGB = prioritizedFreeDiskGB;
     }
 
     @VisibleForTesting
@@ -415,7 +419,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
         azByExistingReplicas.put(azToNumReplicas.get(e.getKey()), new AzWithNodes(e.getKey(), e.getValue()));
       }
 
-      CoresAndDiskComparator coresAndDiskComparator = new CoresAndDiskComparator(attrValues, coresOnNodes, deprioritizedFreeDiskGB);
+      CoresAndDiskComparator coresAndDiskComparator = new CoresAndDiskComparator(attrValues, coresOnNodes, prioritizedFreeDiskGB);
 
       // Now we have for each AZ on which we might have a chance of placing a replica, the list of candidate nodes for replicas
       // (candidate: does not already have a replica of this shard and is in the corresponding AZ).
@@ -488,7 +492,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
     static class CoresAndDiskComparator implements Comparator<Node> {
       private final AttributeValues attrValues;
       private final Map<Node, Integer> coresOnNodes;
-      private final long deprioritizedFreeDiskGB;
+      private final long prioritizedFreeDiskGB;
 
 
       /**
@@ -498,17 +502,17 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
        * attrValues corresponding to the number of cores per node are the initial values, but we want to compare the actual
        * value taking into account placement decisions already made during the current execution of the placement plugin.
        */
-      CoresAndDiskComparator(AttributeValues attrValues, Map<Node, Integer> coresOnNodes, long deprioritizedFreeDiskGB) {
+      CoresAndDiskComparator(AttributeValues attrValues, Map<Node, Integer> coresOnNodes, long prioritizedFreeDiskGB) {
         this.attrValues = attrValues;
         this.coresOnNodes = coresOnNodes;
-        this.deprioritizedFreeDiskGB = deprioritizedFreeDiskGB;
+        this.prioritizedFreeDiskGB = prioritizedFreeDiskGB;
       }
 
       @Override
       public int compare(Node a, Node b) {
         // Note all nodes do have free disk defined. This has been verified earlier.
-        boolean aHasLowFreeSpace = attrValues.getFreeDisk(a).get() < deprioritizedFreeDiskGB;
-        boolean bHasLowFreeSpace = attrValues.getFreeDisk(b).get() < deprioritizedFreeDiskGB;
+        boolean aHasLowFreeSpace = attrValues.getFreeDisk(a).get() < prioritizedFreeDiskGB;
+        boolean bHasLowFreeSpace = attrValues.getFreeDisk(b).get() < prioritizedFreeDiskGB;
         if (aHasLowFreeSpace != bHasLowFreeSpace) {
           // A node with low free space should be considered > node with high free space since it needs to come later in sort order
           return Boolean.compare(aHasLowFreeSpace, bHasLowFreeSpace);
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 3869249..a5d71e7 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
@@ -142,12 +142,6 @@ public class AffinityPlacementFactoryTest extends SolrTestCaseJ4 {
     PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
     // 2 shards, 6 replicas
     assertEquals(12, pp.getReplicaPlacements().size());
-//        List<ReplicaPlacement> placements = new ArrayList<>(pp.getReplicaPlacements());
-//        Collections.sort(placements, Comparator
-//            .comparing((ReplicaPlacement p) -> p.getNode().getName())
-//            .thenComparing((ReplicaPlacement p) -> p.getShardName())
-//            .thenComparing((ReplicaPlacement p) -> p.getReplicaType())
-//        );
     // shard -> AZ -> replica count
     Map<Replica.ReplicaType, Map<String, Map<String, AtomicInteger>>> replicas = new HashMap<>();
     AttributeValues attributeValues = attributeFetcher.fetchAttributes();
@@ -240,6 +234,46 @@ public class AffinityPlacementFactoryTest extends SolrTestCaseJ4 {
   }
 
   @Test
+  public void testFreeDiskConstraints() throws Exception {
+    String collectionName = "testCollection";
+    int NUM_NODES = 3;
+    Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(NUM_NODES);
+    Node smallNode = null;
+    for (int i = 0; i < NUM_NODES; i++) {
+      Builders.NodeBuilder nodeBuilder = clusterBuilder.getNodeBuilders().get(i);
+      nodeBuilder.setCoreCount(0);
+      if (i == 0) {
+        // default minimalFreeDiskGB == 20
+        nodeBuilder.setFreeDiskGB(1L);
+        smallNode = nodeBuilder.build();
+      } else {
+        nodeBuilder.setFreeDiskGB(100L);
+      }
+    }
+
+    Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
+    collectionBuilder.initializeShardsReplicas(2, 0, 0, 0, clusterBuilder.getNodeBuilders());
+    clusterBuilder.addCollection(collectionBuilder);
+
+    Cluster cluster = clusterBuilder.build();
+
+    SolrCollection solrCollection = cluster.getCollection(collectionName);
+
+    PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
+        StreamSupport.stream(solrCollection.shards().spliterator(), false)
+            .map(Shard::getShardName).collect(Collectors.toSet()),
+        cluster.getLiveNodes(), 2, 0, 2);
+
+    PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+    AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
+    PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+    assertEquals(8, pp.getReplicaPlacements().size());
+    for (ReplicaPlacement rp : pp.getReplicaPlacements()) {
+      assertFalse("should not put any replicas on " + smallNode, rp.getNode().equals(smallNode));
+    }
+  }
+
+  @Test
   //@Ignore
   public void testScalability() throws Exception {
     log.info("==== numNodes ====");

[lucene] 11/33: SOLR-15004: Extend the NodeBuilder, move testAffinity to use the Builders.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit cc52210997d5db75dc5a60540a6365b42d610824
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue Nov 24 13:43:11 2020 +0100

    SOLR-15004: Extend the NodeBuilder, move testAffinity to use the Builders.
---
 .../impl/AffinityPlacementFactoryTest.java         | 43 ++++++++++----------
 .../solr/cluster/placement/impl/Builders.java      | 46 ++++++++++++++++++++--
 2 files changed, 65 insertions(+), 24 deletions(-)

diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
index 77f31bf..6b64864 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
@@ -20,6 +20,7 @@ package org.apache.solr.cluster.placement.impl;
 import org.apache.solr.cluster.Cluster;
 import org.apache.solr.cluster.Node;
 import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.Shard;
 import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.*;
 import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
@@ -33,6 +34,8 @@ import java.lang.invoke.MethodHandles;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
 
 /**
  * Unit test for {@link AffinityPlacementFactory}
@@ -153,38 +156,36 @@ public class AffinityPlacementFactoryTest extends Assert {
     @Test
     public void testAvailabilityZones() throws Exception {
         String collectionName = "testCollection";
-
         int NUM_NODES = 6;
-        final Set<Node> liveNodes = new HashSet<>();
-        final Map<Node, Long> nodeToFreeDisk = new HashMap<>();
-        final Map<Node, Integer> nodeToCoreCount = new HashMap<>();
-        final Map<String, Map<Node, String>> zones = Map.of(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP, new HashMap<>());
-        final Map<Node, String> sysprops = zones.get(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP);
+        Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeNodes(NUM_NODES);
         for (int i = 0; i < NUM_NODES; i++) {
-            Node node = new ClusterAbstractionsForTest.NodeImpl("node_" + i);
-            liveNodes.add(node);
-            nodeToFreeDisk.put(node, 100L);
-            nodeToCoreCount.put(node, 0);
+            Builders.NodeBuilder nodeBuilder = clusterBuilder.getNodeBuilders().get(i);
+            nodeBuilder.setCoreCount(0);
+            nodeBuilder.setFreeDiskGB(100L);
             if (i < NUM_NODES / 2) {
-                sysprops.put(node, "az1");
+                nodeBuilder.setSysprop(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP, "az1");
             } else {
-                sysprops.put(node, "az2");
+                nodeBuilder.setSysprop(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP, "az2");
             }
         }
 
-        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = PluginTestHelper.createCollection(collectionName,
-            Map.of(), 2, 0, 0, 0, liveNodes);
-        ClusterAbstractionsForTest.ClusterImpl cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, Map.of());
+        Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
+        collectionBuilder.initializeShardsReplicas(2, 0, 0, 0, clusterBuilder.getNodeBuilders());
+        clusterBuilder.addCollection(collectionBuilder);
+
+        Cluster cluster = clusterBuilder.build();
 
-        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, solrCollection.getShardNames(), liveNodes, 2, 2, 2);
+        SolrCollection solrCollection = cluster.getCollection(collectionName);
+
+        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
+            StreamSupport.stream(solrCollection.shards().spliterator(), false)
+                 .map(Shard::getShardName).collect(Collectors.toSet()),
+            cluster.getLiveNodes(), 2, 2, 2);
 
-        AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(),
-            zones, Map.of());
-        AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
         PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
 
-        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
-        // 2 shards, 5 replicas
+        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, clusterBuilder.buildAttributeFetcher(), placementPlanFactory);
+        // 2 shards, 6 replicas
         assertEquals(12, pp.getReplicaPlacements().size());
         List<ReplicaPlacement> placements = new ArrayList<>(pp.getReplicaPlacements());
         Collections.sort(placements, Comparator
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
index d954dd1..996186f 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/Builders.java
@@ -68,6 +68,8 @@ public class Builders {
         AttributeFetcher buildAttributeFetcher() {
             Map<Node, Integer> nodeToCoreCount = new HashMap<>();
             Map<Node, Long> nodeToFreeDisk = new HashMap<>();
+            Map<String, Map<Node, String>> sysprops = new HashMap<>();
+            Map<String, Map<Node, Double>> metrics = new HashMap<>();
 
             // TODO And a few more missing and will be added...
 
@@ -82,9 +84,21 @@ public class Builders {
                 if (nodeBuilder.getFreeDiskGB() != null) {
                     nodeToFreeDisk.put(node, nodeBuilder.getFreeDiskGB());
                 }
+                if (nodeBuilder.getSysprops() != null) {
+                    nodeBuilder.getSysprops().forEach((name, value) -> {
+                        sysprops.computeIfAbsent(name, n -> new HashMap<>())
+                            .put(node, value);
+                    });
+                }
+                if (nodeBuilder.getMetrics() != null) {
+                    nodeBuilder.getMetrics().forEach((name, value) -> {
+                        metrics.computeIfAbsent(name, n -> new HashMap<>())
+                            .put(node, value);
+                    });
+                }
             }
 
-            AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
+            AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), sysprops, metrics);
             return new AttributeFetcherForTest(attributeValues);
         }
     }
@@ -116,7 +130,7 @@ public class Builders {
             shardBuilders = new LinkedList<>();
 
             for (int s = 0; s < countShards; s++) {
-                String shardName = collectionName + "_s" + s;
+                String shardName = "shard" + (s + 1);
 
                 LinkedList<ReplicaBuilder> replicas = new LinkedList<>();
                 ReplicaBuilder leader = null;
@@ -130,7 +144,7 @@ public class Builders {
                 for (Pair<Replica.ReplicaType, Integer> tc : replicaTypes) {
                     Replica.ReplicaType type = tc.first();
                     int count = tc.second();
-                    String replicaPrefix = shardName + "_" + type.name() + "_";
+                    String replicaPrefix = collectionName + "_" + shardName + "_replica_" + type.name().toLowerCase(Locale.ROOT).charAt(0);
                     for (int r = 0; r < count; r++) {
                         String replicaName = replicaPrefix + r;
                         String coreName = replicaName + "_c";
@@ -255,6 +269,8 @@ public class Builders {
         private String nodeName = null;
         private Integer coreCount = null;
         private Long freeDiskGB = null;
+        private Map<String, String> sysprops = null;
+        private Map<String, Double> metrics = null;
 
         NodeBuilder setNodeName(String nodeName) {
             this.nodeName = nodeName;
@@ -271,6 +287,22 @@ public class Builders {
             return this;
         }
 
+        NodeBuilder setSysprop(String key, String value) {
+            if (sysprops == null) {
+                sysprops = new HashMap<>();
+            }
+            sysprops.put(key, value);
+            return this;
+        }
+
+        NodeBuilder setMetric(String key, Double value) {
+            if (metrics == null) {
+                metrics = new HashMap<>();
+            }
+            metrics.put(key, value);
+            return this;
+        }
+
         Integer getCoreCount() {
             return coreCount;
         }
@@ -279,6 +311,14 @@ public class Builders {
             return freeDiskGB;
         }
 
+        Map<String, String> getSysprops() {
+            return sysprops;
+        }
+
+        Map<String, Double> getMetrics() {
+            return metrics;
+        }
+
         Node build() {
             // It is ok to build a new instance each time, that instance does the right thing with equals() and hashCode()
             return new ClusterAbstractionsForTest.NodeImpl(nodeName);

[lucene] 27/33: SOLR-15016: Improve config hierarchy. Cleanup. Fix the discovery of config implementation class.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

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();
   }
 }

[lucene] 08/33: SOLR-15004: Start testing the availability zone affinity. Add toString() to make it easier to debug / report. Improve test abstractions to make it easier to set up tests.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 9ca59200949ea3d578e30d52066f86072814beb8
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Mon Nov 23 19:42:01 2020 +0100

    SOLR-15004: Start testing the availability zone affinity. Add toString() to make it
    easier to debug / report. Improve test abstractions to make it easier to set up tests.
---
 .../cluster/placement/impl/PlacementPlanImpl.java  | 10 +++
 .../placement/impl/ReplicaPlacementImpl.java       |  5 ++
 .../plugins/AffinityPlacementFactory.java          | 42 ++++++------
 .../impl/AffinityPlacementFactoryTest.java         | 79 ++++++++++++++++++----
 .../placement/impl/ClusterAbstractionsForTest.java | 43 ++++++++++--
 .../cluster/placement/impl/PluginTestHelper.java   | 39 +++++++++--
 6 files changed, 174 insertions(+), 44 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPlanImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPlanImpl.java
index 2dde07b..a8a65e0 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPlanImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPlanImpl.java
@@ -42,4 +42,14 @@ class PlacementPlanImpl implements PlacementPlan {
   public Set<ReplicaPlacement> getReplicaPlacements() {
     return replicaPlacements;
   }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("PlacementPlan{");
+    for (ReplicaPlacement placement : replicaPlacements) {
+      sb.append("\n").append(placement.toString());
+    }
+    sb.append("\n}");
+    return sb.toString();
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/ReplicaPlacementImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/ReplicaPlacementImpl.java
index 0bf7564..69d9718 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/ReplicaPlacementImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/ReplicaPlacementImpl.java
@@ -60,6 +60,11 @@ class ReplicaPlacementImpl implements ReplicaPlacement {
     return replicaType;
   }
 
+  @Override
+  public String toString() {
+    return solrCollection.getName() + "/" + shardName + "/" + replicaType + "->" + node.getName();
+  }
+
   /**
    * Translates a set of {@link ReplicaPlacement} returned by a plugin into a list of {@link ReplicaPosition} expected
    * by {@link org.apache.solr.cloud.api.collections.Assign.AssignStrategy}
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 50ecb8d..9a00903 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
@@ -118,6 +118,26 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   /**
+   * <p>Name of the system property on a node indicating which (public cloud) Availability Zone that node is in. The value
+   * is any string, different strings denote different availability zones.
+   *
+   * <p>Nodes on which this system property is not defined are considered being in the same Availability Zone
+   * {@link #UNDEFINED_AVAILABILITY_ZONE} (hopefully the value of this constant is not the name of a real Availability Zone :).
+   */
+  public static final String AVAILABILITY_ZONE_SYSPROP = "availability_zone";
+
+  /**
+   * <p>Name of the system property on a node indicating the type of replicas allowed on that node.
+   * The value of that system property is a comma separated list or a single string of value names of
+   * {@link Replica.ReplicaType} (case insensitive). If that property is not defined, that node is
+   * considered accepting all replica types (i.e. undefined is equivalent to {@code "NRT,Pull,tlog"}).
+   */
+  public static final String REPLICA_TYPE_SYSPROP = "replica_type";
+
+  /** This is the "AZ" name for nodes that do not define an AZ. Should not match a real AZ name (I think we're safe) */
+  public static final String UNDEFINED_AVAILABILITY_ZONE = "uNd3f1NeD";
+
+  /**
    * 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
    * having to call a constructor with more parameters (if we were to instantiate the plugin class directly without going
@@ -138,26 +158,6 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
    * on what the plugin does.
    */
   static private class AffinityPlacementPlugin implements PlacementPlugin {
-    /**
-     * <p>Name of the system property on a node indicating which (public cloud) Availability Zone that node is in. The value
-     * is any string, different strings denote different availability zones.
-     *
-     * <p>Nodes on which this system property is not defined are considered being in the same Availability Zone
-     * {@link #UNDEFINED_AVAILABILITY_ZONE} (hopefully the value of this constant is not the name of a real Availability Zone :).
-     */
-    public static final String AVAILABILITY_ZONE_SYSPROP = "availability_zone";
-    /** This is the "AZ" name for nodes that do not define an AZ. Should not match a real AZ name (I think we're safe) */
-    public static final String UNDEFINED_AVAILABILITY_ZONE = "uNd3f1NeD";
-
-    /**
-     * <p>Name of the system property on a node indicating the type of replicas allowed on that node.
-     * The value of that system property is a comma separated list or a single string of value names of
-     * {@link org.apache.solr.cluster.Replica.ReplicaType} (case insensitive). If that property is not defined, that node is
-     * considered accepting all replica types (i.e. undefined is equivalent to {@code "NRT,Pull,tlog"}).
-     *
-     * <p>See {@link #getNodesPerReplicaType}.
-     */
-    public static final String REPLICA_TYPE_SYSPROP = "replica_type";
 
     /**
      * If a node has strictly less GB of free disk than this value, the node is excluded from assignment decisions.
@@ -486,7 +486,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory {
        * The data we sort on is not part of the {@link Node} instances but has to be retrieved from the attributes and configuration.
        * The number of cores per node is passed in a map whereas the free disk is fetched from the attributes due to the
        * fact that we update the number of cores per node as we do allocations, but we do not update the free disk. The
-       * attrValues correpsonding to the number of cores per node are the initial values, but we want to comapre the actual
+       * attrValues corresponding to the number of cores per node are the initial values, but we want to compare the actual
        * value taking into account placement decisions already made during the current execution of the placement plugin.
        */
       CoresAndDiskComparator(AttributeValues attrValues, Map<Node, Integer> coresOnNodes, long deprioritizedFreeDiskGB) {
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
index 80f30105..bcfc233 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
@@ -20,21 +20,26 @@ package org.apache.solr.cluster.placement.impl;
 import org.apache.solr.cluster.Cluster;
 import org.apache.solr.cluster.Node;
 import org.apache.solr.cluster.Replica;
-import org.apache.solr.cluster.Shard;
 import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.*;
 import org.apache.solr.cluster.placement.plugins.AffinityPlacementFactory;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -75,25 +80,22 @@ public class AffinityPlacementFactoryTest extends Assert {
         Node node2 = new ClusterAbstractionsForTest.NodeImpl("node2");
         Set<Node> liveNodes = Set.of(node1, node2);
 
-        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(collectionName, Map.of());
+        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection;
         // Make sure new collections are not visible in the cluster state and existing ones are
         final Map<String, SolrCollection> clusterCollections;
-        final Map<String, Shard> shards;
         if (hasExistingCollection) {
             // An existing collection with a single replica on node 1. Note that new collections already exist by the time the plugin is called, but are empty
-            shards = PluginTestHelper.createShardsAndReplicas(solrCollection, 1, 1, Set.of(node1));
-            solrCollection.setShards(shards);
+            solrCollection = PluginTestHelper.createCollection(collectionName, Map.of(), 1, 1, 0, 0, Set.of(node1));
             clusterCollections = Map.of(solrCollection.getName(), solrCollection);
         } else {
             // A new collection has the shards defined ok but no replicas
-            shards = PluginTestHelper.createShardsAndReplicas(solrCollection, 1, 0, Set.of());
-            solrCollection.setShards(shards);
+            solrCollection = PluginTestHelper.createCollection(collectionName, Map.of(), 1, 0, 0, 0, Set.of());
             clusterCollections = Map.of();
         }
 
         Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, clusterCollections);
         // Place a new replica for the (only) existing shard of the collection
-        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, Set.of(shards.keySet().iterator().next()), liveNodes, 1, 0, 0);
+        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, Set.of(solrCollection.shards().iterator().next().getShardName()), liveNodes, 1, 0, 0);
         // More cores on node2
         Map<Node, Integer> nodeToCoreCount = Map.of(node1, 1, node2, 10);
         // A lot of free disk on the two nodes
@@ -111,6 +113,54 @@ public class AffinityPlacementFactoryTest extends Assert {
     }
 
     @Test
+    public void testAvailabilityZones() throws Exception {
+        String collectionName = "testCollection";
+
+        int NUM_NODES = 6;
+        final Set<Node> liveNodes = new HashSet<>();
+        final Map<Node, Long> nodeToFreeDisk = new HashMap<>();
+        final Map<Node, Integer> nodeToCoreCount = new HashMap<>();
+        final Map<String, Map<Node, String>> zones = Map.of(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP, new HashMap<>());
+        final Map<Node, String> sysprops = zones.get(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP);
+        for (int i = 0; i < NUM_NODES; i++) {
+            Node node = new ClusterAbstractionsForTest.NodeImpl("node_" + i);
+            liveNodes.add(node);
+            nodeToFreeDisk.put(node, 100L);
+            nodeToCoreCount.put(node, 0);
+            if (i < NUM_NODES / 2) {
+                sysprops.put(node, "az1");
+            } else {
+                sysprops.put(node, "az2");
+            }
+        }
+
+        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = PluginTestHelper.createCollection(collectionName,
+            Map.of(), 2, 0, 0, 0, liveNodes);
+        ClusterAbstractionsForTest.ClusterImpl cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, Map.of());
+
+        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, solrCollection.getShardNames(), liveNodes, 2, 2, 2);
+
+        AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(),
+            zones, Map.of());
+        AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
+        PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+
+        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+        // 2 shards, 5 replicas
+        assertEquals(12, pp.getReplicaPlacements().size());
+        List<ReplicaPlacement> placements = new ArrayList<>(pp.getReplicaPlacements());
+        Collections.sort(placements, Comparator
+            .comparing((ReplicaPlacement p) -> p.getNode().getName())
+            .thenComparing((ReplicaPlacement p) -> p.getShardName())
+            .thenComparing((ReplicaPlacement p) -> p.getReplicaType())
+        );
+        log.info(placements.toString());
+        // AZ -> shard -> replica count
+        //Map<String, Map<String, AtomicInteger>>
+    }
+
+    @Test
+    //@Ignore
     public void testScalability() throws Exception {
         log.info("==== numNodes ====");
         runTestScalability(1000, 100, 40, 40, 20);
@@ -150,13 +200,16 @@ public class AffinityPlacementFactoryTest extends Assert {
             nodeToFreeDisk.put(node, Long.valueOf(numNodes));
             nodeToCoreCount.put(node, 0);
         }
-        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(collectionName, Map.of());
-        Map<String, Shard> shards = PluginTestHelper.createShardsAndReplicas(solrCollection, numShards, 0, Set.of());
-        solrCollection.setShards(shards);
+        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection =
+            PluginTestHelper.createCollection(collectionName, Map.of(), numShards, 0, 0, 0, Set.of());
 
         Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, Map.of());
-        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, shards.keySet(), liveNodes,
-            nrtReplicas, tlogReplicas, pullReplicas);
+        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection,
+            // XXX awkward!
+            // StreamSupport.stream(solrCollection.shards().spliterator(), false)
+            //     .map(Shard::getShardName).collect(Collectors.toSet()),
+            solrCollection.getShardNames(),
+            liveNodes, nrtReplicas, tlogReplicas, pullReplicas);
 
         AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
         AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
index 188e3c3..02395d6 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
@@ -30,12 +30,16 @@ import java.util.stream.Collectors;
 class ClusterAbstractionsForTest {
 
     static class ClusterImpl implements Cluster {
-        private final Set<Node> liveNodes;
-        private final Map<String, SolrCollection> collections;
+        private final Set<Node> liveNodes = new HashSet<>();
+        private final Map<String, SolrCollection> collections = new HashMap<>();
+
+        ClusterImpl() {
+
+        }
 
         ClusterImpl(Set<Node> liveNodes, Map<String, SolrCollection> collections) throws IOException {
-            this.liveNodes = liveNodes;
-            this.collections = collections;
+            this.liveNodes.addAll(liveNodes);
+            this.collections.putAll(collections);
         }
 
         @Override
@@ -58,6 +62,33 @@ class ClusterAbstractionsForTest {
         public Iterable<SolrCollection> collections() {
             return ClusterImpl.this::iterator;
         }
+
+        // for unit tests
+
+        ClusterImpl addNode(Node node) {
+            liveNodes.add(node);
+            return this;
+        }
+
+        ClusterImpl removeNode(Node node) {
+            liveNodes.remove(node);
+            return this;
+        }
+
+        ClusterImpl putCollection(SolrCollection collection) {
+            collections.put(collection.getName(), collection);
+            return this;
+        }
+
+        ClusterImpl removeCollection(String name) {
+            collections.remove(name);
+            return this;
+        }
+
+        ClusterImpl removeAllCollections() {
+            collections.clear();
+            return this;
+        }
     }
 
 
@@ -122,6 +153,10 @@ class ClusterAbstractionsForTest {
             this.shards = shards;
         }
 
+        Set<String> getShardNames() {
+            return shards.keySet();
+        }
+
         @Override
         public String getName() {
             return collectionName;
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
index 13ef70d..f8b4d87 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
@@ -29,24 +29,49 @@ import java.util.Set;
 
 public class PluginTestHelper {
 
+    static ClusterAbstractionsForTest.SolrCollectionImpl createCollection(String name, Map<String, String> properties,
+                                           int numShards, int nrtReplicas, int tlogReplicas, int pullReplicas, Set<Node> nodes) {
+        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(name, properties);
+        Map<String, Shard> shards = createShardsAndReplicas(solrCollection, numShards, nrtReplicas, tlogReplicas, pullReplicas, nodes);
+        solrCollection.setShards(shards);
+        return solrCollection;
+    }
+
     /**
      * Builds the representation of shards for a collection, based on the number of shards and replicas for each to create.
      * The replicas are allocated to the provided nodes in a round robin way. The leader is set to the last replica of each shard.
      */
-    static Map<String, Shard> createShardsAndReplicas(SolrCollection collection, int numShards, int numNrtReplicas, Set<Node> nodes) {
+    static Map<String, Shard> createShardsAndReplicas(SolrCollection collection, int numShards,
+                                                      int nrtReplicas, int tlogReplicas, int pullReplicas,
+                                                      Set<Node> nodes) {
         Iterator<Node> nodeIterator = nodes.iterator();
 
         Map<String, Shard> shards = new HashMap<>();
 
         for (int s = 0; s < numShards; s++) {
-            String shardName = collection.getName() + "_s" + s;
+            // "traditional" shard name
+            String shardName = "shard" + (s + 1);
 
             ClusterAbstractionsForTest.ShardImpl shard = new ClusterAbstractionsForTest.ShardImpl(shardName, collection, Shard.ShardState.ACTIVE);
 
             Map<String, Replica> replicas = new HashMap<>();
+
             Replica leader = null;
-            for (int r = 0; r < numNrtReplicas; r++) {
-                String replicaName = shardName + "_r" + r;
+            int totalReplicas = nrtReplicas + tlogReplicas + pullReplicas;
+            for (int r = 0; r < totalReplicas; r++) {
+                Replica.ReplicaType type;
+                String suffix;
+                if (r < nrtReplicas) {
+                    type = Replica.ReplicaType.NRT;
+                    suffix = "n";
+                } else if (r < nrtReplicas + tlogReplicas) {
+                    type = Replica.ReplicaType.TLOG;
+                    suffix = "t";
+                } else {
+                    type = Replica.ReplicaType.PULL;
+                    suffix = "p";
+                }
+                String replicaName = shardName + "_replica_" + suffix + r;
                 String coreName = replicaName + "_c";
                 final Node node;
                 if (!nodeIterator.hasNext()) {
@@ -55,10 +80,12 @@ public class PluginTestHelper {
                 // If the nodes set is empty, this call will fail
                 node = nodeIterator.next();
 
-                Replica replica = new ClusterAbstractionsForTest.ReplicaImpl(replicaName, coreName, shard, Replica.ReplicaType.NRT, Replica.ReplicaState.ACTIVE, node);
+                Replica replica = new ClusterAbstractionsForTest.ReplicaImpl(replicaName, coreName, shard, type, Replica.ReplicaState.ACTIVE, node);
 
                 replicas.put(replica.getReplicaName(), replica);
-                leader = replica;
+                if (replica.getType() == Replica.ReplicaType.NRT) {
+                    leader = replica;
+                }
             }
 
             shard.setReplicas(replicas, leader);

[lucene] 33/33: SOLR-15016: CHANGES.txt.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 6d9e22c59867820700be838b0536f5a66aeb12e2
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Sat Dec 5 20:23:38 2020 +0100

    SOLR-15016: CHANGES.txt.
---
 solr/CHANGES.txt | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index dcb2a31..de50aea 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -178,6 +178,8 @@ Other Changes
 * SOLR-14789: Docker: Migrate docker image creation from docker-solr repo to solr/docker.
   (Houston Putman, Martijn Koster, Tim Potter, David Smiley, janhoy, Mike Drob)
 
+* SOLR-15016: Replica placement plugins should use container plugins API / configs. (ab, ilan)
+
 Bug Fixes
 ---------------------
 * SOLR-14546: Fix for a relatively hard to hit issue in OverseerTaskProcessor that could lead to out of order execution

[lucene] 28/33: SOLR-15016: Fix issues found in review.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 9a2865b94045b061547bf85e57a47627f4889e45
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Dec 3 16:58:51 2020 +0100

    SOLR-15016: Fix issues found in review.
---
 .../apache/solr/api/ContainerPluginsRegistry.java  |  6 ++---
 .../apache/solr/cloud/api/collections/Assign.java  |  2 ++
 .../impl/CollectionsRepairEventListener.java       | 29 +++++++++-------------
 .../cluster/placement/PlacementPluginFactory.java  |  2 ++
 4 files changed, 18 insertions(+), 21 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 453b092..b9cc6c7 100644
--- a/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
+++ b/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
@@ -58,7 +58,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.lucene.util.IOUtils.closeWhileHandlingException;
-import static org.apache.solr.common.util.Utils.makeMap;
 
 /**
  * This class manages the container-level plugins and their Api-s. It is
@@ -143,7 +142,7 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
   }
   @SuppressWarnings("unchecked")
   public synchronized void refresh() {
-    Map<String, Object> pluginInfos = null;
+    Map<String, Object> pluginInfos;
     try {
       pluginInfos = ContainerPluginsApi.plugins(coreContainer.zkClientSupplier);
     } catch (IOException e) {
@@ -236,9 +235,8 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
     return path;
   }
 
-  @SuppressWarnings({"rawtypes", "unchecked"})
   private static  Map<String, String> getTemplateVars(PluginMeta pluginMeta) {
-    return (Map) makeMap("plugin-name", pluginMeta.name, "path-prefix", pluginMeta.pathPrefix);
+    return Map.of("plugin-name", pluginMeta.name, "path-prefix", pluginMeta.pathPrefix);
   }
 
   private static class ApiHolder extends Api {
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 0c249c9..786bfa9 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
@@ -492,6 +492,8 @@ public class Assign {
   /**
    * Creates the appropriate instance of {@link AssignStrategy} based on how the cluster and/or individual collections are
    * configured.
+   * <p>If {@link PlacementPlugin} instance is null this call will return {@link LegacyAssignStrategy}, otherwise
+   * {@link PlacementPluginAssignStrategy} will be used.</p>
    */
   public static AssignStrategy createAssignStrategy(PlacementPlugin placementPlugin, ClusterState clusterState, DocCollection collection) {
     if (placementPlugin != null) {
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 fd5f610..8984d1d 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,8 +41,8 @@ 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.PlacementPluginConfig;
 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;
@@ -78,7 +78,7 @@ public class CollectionsRepairEventListener implements ClusterEventListener, Clu
   private int waitForSecond = DEFAULT_WAIT_FOR_SEC;
 
   private ScheduledThreadPoolExecutor waitForExecutor;
-  private PlacementPluginFactory<? extends MapWriter> placementPluginFactory;
+  private final PlacementPluginFactory<? extends PlacementPluginConfig> placementPluginFactory;
 
   public CollectionsRepairEventListener(CoreContainer cc) {
     this.solrClient = cc.getSolrClientCache().getCloudSolrClient(cc.getZkController().getZkClient().getZkServerAddress());
@@ -114,7 +114,7 @@ public class CollectionsRepairEventListener implements ClusterEventListener, Clu
     }
   }
 
-  private Map<String, Long> nodeNameVsTimeRemoved = new ConcurrentHashMap<>();
+  private final Map<String, Long> nodeNameVsTimeRemoved = new ConcurrentHashMap<>();
 
   private void handleNodesDown(NodesDownEvent event) {
 
@@ -125,9 +125,7 @@ public class CollectionsRepairEventListener implements ClusterEventListener, Clu
     Set<String> trackingKeySet = nodeNameVsTimeRemoved.keySet();
     trackingKeySet.removeAll(solrCloudManager.getClusterStateProvider().getLiveNodes());
     // add any new lost nodes (old lost nodes are skipped)
-    event.getNodeNames().forEachRemaining(lostNode -> {
-      nodeNameVsTimeRemoved.computeIfAbsent(lostNode, n -> solrCloudManager.getTimeSource().getTimeNs());
-    });
+    event.getNodeNames().forEachRemaining(lostNode -> nodeNameVsTimeRemoved.computeIfAbsent(lostNode, n -> solrCloudManager.getTimeSource().getTimeNs()));
   }
 
   private void runRepair() {
@@ -195,7 +193,6 @@ public class CollectionsRepairEventListener implements ClusterEventListener, Clu
             newPositions.put(coll.getName(), positions);
           } catch (Exception e) {
             log.warn("Exception computing positions for {}/{}: {}", coll.getName(), shard, e);
-            return;
           }
         });
       });
@@ -210,15 +207,13 @@ public class CollectionsRepairEventListener implements ClusterEventListener, Clu
     // send ADDREPLICA admin requests for each lost replica
     // XXX should we use 'async' for that, to avoid blocking here?
     List<CollectionAdminRequest.AddReplica> addReplicas = new ArrayList<>();
-    newPositions.forEach((collection, positions) -> {
-      positions.forEach(position -> {
-        CollectionAdminRequest.AddReplica addReplica = CollectionAdminRequest
-            .addReplicaToShard(collection, position.shard, position.type);
-        addReplica.setNode(position.node);
-        addReplica.setAsyncId(ASYNC_ID_PREFIX + counter.incrementAndGet());
-        addReplicas.add(addReplica);
-      });
-    });
+    newPositions.forEach((collection, positions) -> positions.forEach(position -> {
+      CollectionAdminRequest.AddReplica addReplica = CollectionAdminRequest
+          .addReplicaToShard(collection, position.shard, position.type);
+      addReplica.setNode(position.node);
+      addReplica.setAsyncId(ASYNC_ID_PREFIX + counter.incrementAndGet());
+      addReplicas.add(addReplica);
+    }));
     addReplicas.forEach(addReplica -> {
       try {
         solrClient.request(addReplica);
@@ -235,7 +230,7 @@ public class CollectionsRepairEventListener implements ClusterEventListener, Clu
         new SolrNamedThreadFactory("collectionsRepair_waitFor"));
     waitForExecutor.setRemoveOnCancelPolicy(true);
     waitForExecutor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
-    waitForExecutor.scheduleAtFixedRate(() -> runRepair(), 0, waitForSecond, TimeUnit.SECONDS);
+    waitForExecutor.scheduleAtFixedRate(this::runRepair, 0, waitForSecond, TimeUnit.SECONDS);
     state = State.RUNNING;
   }
 
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 bed9ea9..1358a93 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
@@ -36,6 +36,8 @@ public interface PlacementPluginFactory<T extends PlacementPluginConfig> extends
    * 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).
+   * <p>If this method returns null then a simple legacy assignment strategy will be used
+   * (see {@link org.apache.solr.cloud.api.collections.Assign.LegacyAssignStrategy}).</p>
    */
   PlacementPlugin createPluginInstance();
 

[lucene] 10/33: Merge branch 'jira/solr-15004' of https://github.com/apache/lucene-solr into jira/solr-15004 (and make precommit happy)

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 0a112f5ffb7836a3a7ea944e89c43b21f84301b7
Merge: bd63ad1 9ca5920
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Tue Nov 24 02:12:16 2020 +0100

    Merge branch 'jira/solr-15004' of https://github.com/apache/lucene-solr into jira/solr-15004 (and make precommit happy)

 .../cluster/placement/impl/PlacementPlanImpl.java  | 10 +++
 .../placement/impl/ReplicaPlacementImpl.java       |  5 ++
 .../plugins/AffinityPlacementFactory.java          | 42 ++++++-------
 .../impl/AffinityPlacementFactoryTest.java         | 73 ++++++++++++++++++----
 .../placement/impl/ClusterAbstractionsForTest.java | 40 ++++++++++--
 .../cluster/placement/impl/PluginTestHelper.java   | 39 ++++++++++--
 6 files changed, 164 insertions(+), 45 deletions(-)

diff --cc solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java
index 50ecb8d,9a00903..6982b8f
--- 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
@@@ -118,6 -118,26 +118,26 @@@ public class AffinityPlacementFactory i
    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
  
    /**
+    * <p>Name of the system property on a node indicating which (public cloud) Availability Zone that node is in. The value
+    * is any string, different strings denote different availability zones.
+    *
+    * <p>Nodes on which this system property is not defined are considered being in the same Availability Zone
+    * {@link #UNDEFINED_AVAILABILITY_ZONE} (hopefully the value of this constant is not the name of a real Availability Zone :).
+    */
+   public static final String AVAILABILITY_ZONE_SYSPROP = "availability_zone";
+ 
+   /**
+    * <p>Name of the system property on a node indicating the type of replicas allowed on that node.
+    * The value of that system property is a comma separated list or a single string of value names of
 -   * {@link Replica.ReplicaType} (case insensitive). If that property is not defined, that node is
++   * {@link org.apache.solr.cluster.Replica.ReplicaType} (case insensitive). If that property is not defined, that node is
+    * considered accepting all replica types (i.e. undefined is equivalent to {@code "NRT,Pull,tlog"}).
+    */
+   public static final String REPLICA_TYPE_SYSPROP = "replica_type";
+ 
+   /** This is the "AZ" name for nodes that do not define an AZ. Should not match a real AZ name (I think we're safe) */
+   public static final String UNDEFINED_AVAILABILITY_ZONE = "uNd3f1NeD";
+ 
+   /**
     * 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
     * having to call a constructor with more parameters (if we were to instantiate the plugin class directly without going
diff --cc solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
index bf62c17,bcfc233..77f31bf
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AffinityPlacementFactoryTest.java
@@@ -155,6 -113,54 +151,54 @@@ public class AffinityPlacementFactoryTe
      }
  
      @Test
+     public void testAvailabilityZones() throws Exception {
+         String collectionName = "testCollection";
+ 
+         int NUM_NODES = 6;
+         final Set<Node> liveNodes = new HashSet<>();
+         final Map<Node, Long> nodeToFreeDisk = new HashMap<>();
+         final Map<Node, Integer> nodeToCoreCount = new HashMap<>();
+         final Map<String, Map<Node, String>> zones = Map.of(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP, new HashMap<>());
+         final Map<Node, String> sysprops = zones.get(AffinityPlacementFactory.AVAILABILITY_ZONE_SYSPROP);
+         for (int i = 0; i < NUM_NODES; i++) {
+             Node node = new ClusterAbstractionsForTest.NodeImpl("node_" + i);
+             liveNodes.add(node);
+             nodeToFreeDisk.put(node, 100L);
+             nodeToCoreCount.put(node, 0);
+             if (i < NUM_NODES / 2) {
+                 sysprops.put(node, "az1");
+             } else {
+                 sysprops.put(node, "az2");
+             }
+         }
+ 
+         ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = PluginTestHelper.createCollection(collectionName,
+             Map.of(), 2, 0, 0, 0, liveNodes);
+         ClusterAbstractionsForTest.ClusterImpl cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, Map.of());
+ 
+         PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, solrCollection.getShardNames(), liveNodes, 2, 2, 2);
+ 
+         AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(),
+             zones, Map.of());
+         AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
+         PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+ 
+         PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+         // 2 shards, 5 replicas
+         assertEquals(12, pp.getReplicaPlacements().size());
+         List<ReplicaPlacement> placements = new ArrayList<>(pp.getReplicaPlacements());
+         Collections.sort(placements, Comparator
+             .comparing((ReplicaPlacement p) -> p.getNode().getName())
+             .thenComparing((ReplicaPlacement p) -> p.getShardName())
+             .thenComparing((ReplicaPlacement p) -> p.getReplicaType())
+         );
 -        log.info(placements.toString());
++        log.info(placements.toString());  //nowarn
+         // AZ -> shard -> replica count
+         //Map<String, Map<String, AtomicInteger>>
+     }
+ 
+     @Test
+     //@Ignore
      public void testScalability() throws Exception {
          log.info("==== numNodes ====");
          runTestScalability(1000, 100, 40, 40, 20);
diff --cc solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
index 69d3238,02395d6..da82240
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
@@@ -20,7 -20,7 +20,6 @@@ package org.apache.solr.cluster.placeme
  import org.apache.solr.cluster.*;
  
  import javax.annotation.Nonnull;
--import java.io.IOException;
  import java.util.*;
  import java.util.stream.Collectors;
  
@@@ -30,12 -30,16 +29,12 @@@
  class ClusterAbstractionsForTest {
  
      static class ClusterImpl implements Cluster {
-         private final Set<Node> liveNodes;
-         private final Map<String, SolrCollection> collections;
+         private final Set<Node> liveNodes = new HashSet<>();
+         private final Map<String, SolrCollection> collections = new HashMap<>();
  
 -        ClusterImpl() {
 -
 -        }
 -
 -        ClusterImpl(Set<Node> liveNodes, Map<String, SolrCollection> collections) throws IOException {
 +        ClusterImpl(Set<Node> liveNodes, Map<String, SolrCollection> collections) {
-             this.liveNodes = liveNodes;
-             this.collections = collections;
+             this.liveNodes.addAll(liveNodes);
+             this.collections.putAll(collections);
          }
  
          @Override

[lucene] 31/33: SOLR-15016: Use utility method here to allow for nulls.

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

janhoy pushed a commit to tag history/branches/lucene-solr/jira/solr-15016
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit b5e5dda8844c4ebb05b0738174aadca52da5dcdf
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Sat Dec 5 11:01:32 2020 +0100

    SOLR-15016: Use utility method here to allow for nulls.
---
 solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

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 b9cc6c7..9fa0261 100644
--- a/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
+++ b/solr/core/src/java/org/apache/solr/api/ContainerPluginsRegistry.java
@@ -58,6 +58,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.lucene.util.IOUtils.closeWhileHandlingException;
+import static org.apache.solr.common.util.Utils.makeMap;
 
 /**
  * This class manages the container-level plugins and their Api-s. It is
@@ -235,8 +236,9 @@ public class ContainerPluginsRegistry implements ClusterPropertiesListener, MapW
     return path;
   }
 
+  @SuppressWarnings({"rawtypes", "unchecked"})
   private static  Map<String, String> getTemplateVars(PluginMeta pluginMeta) {
-    return Map.of("plugin-name", pluginMeta.name, "path-prefix", pluginMeta.pathPrefix);
+    return (Map) makeMap("plugin-name", pluginMeta.name, "path-prefix", pluginMeta.pathPrefix);
   }
 
   private static class ApiHolder extends Api {