You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ho...@apache.org on 2023/05/26 14:55:30 UTC

[solr] branch main updated: SOLR-16816: Update node metrics when making affinityPlacement selections (#1661)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 9ad6c679606 SOLR-16816: Update node metrics when making affinityPlacement selections (#1661)
9ad6c679606 is described below

commit 9ad6c679606f7bd9a510afb74b7be21acafee5fe
Author: Houston Putman <ho...@apache.org>
AuthorDate: Fri May 26 10:55:23 2023 -0400

    SOLR-16816: Update node metrics when making affinityPlacement selections (#1661)
    
    Therefore selections can be made given the expected cluster
    information after the previous selections are implemented.
    Initially, this will ensure that replica placements do not
    make Solr Nodes go under their minimum free disk space.
---
 solr/CHANGES.txt                                   |   3 +
 .../solr/cluster/placement/AttributeValues.java    |  12 ++
 .../apache/solr/cluster/placement/NodeMetric.java  |  16 +++
 .../placement/impl/AttributeFetcherImpl.java       |  22 ++-
 .../placement/impl/AttributeValuesImpl.java        |  22 +++
 .../cluster/placement/impl/BuiltInMetrics.java     |  79 +++++++++++
 .../cluster/placement/impl/NodeMetricImpl.java     | 145 +++++++++++++++----
 .../cluster/placement/impl/ReplicaMetricImpl.java  |  11 --
 .../plugins/AffinityPlacementFactory.java          | 154 ++++++++++++++-------
 .../plugins/MinimizeCoresPlacementFactory.java     |   8 +-
 .../impl/PlacementPluginIntegrationTest.java       |  44 +++---
 .../plugins/AffinityPlacementFactoryTest.java      |  83 ++++++++++-
 .../apache/solr/cluster/placement/Builders.java    |  43 ++++--
 .../solr/cluster/placement/BuildersTest.java       |  24 ++--
 14 files changed, 510 insertions(+), 156 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 663ec6c8d43..22ee9677e9a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -144,6 +144,9 @@ Improvements
 * SOLR-9378: Internal shard requests no longer include the wasteful shard.url param.  [shard] transformer now defaults to returning 
   only the shard id (based on luceneMatchVersion), but can be configured to return the legacy list of replicas. (hossman)
 
+* SOLR-16816: Update node metrics while making affinityPlacement selections. Therefore selections can be made given the expected cluster
+  information after the previous selections are implemented. (Houston Putman)
+
 Optimizations
 ---------------------
 
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 dced15dafed..4f367e86524 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
@@ -30,6 +30,18 @@ public interface AttributeValues {
   /** For the given node: metric identified by an instance of {@link NodeMetric} */
   <T> Optional<T> getNodeMetric(Node node, NodeMetric<T> metric);
 
+  /**
+   * For the given node: increase the metric identified by an instance of {@link NodeMetric} by the
+   * given amount
+   */
+  <T> boolean increaseNodeMetric(Node node, NodeMetric<T> metric, T byValue);
+
+  /**
+   * For the given node: decrease the metric identified by an instance of {@link NodeMetric} by the
+   * given amount
+   */
+  <T> boolean decreaseNodeMetric(Node node, NodeMetric<T> metric, T byValue);
+
   /** Get collection metrics. */
   Optional<CollectionMetrics> getCollectionMetrics(String collectionName);
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/NodeMetric.java b/solr/core/src/java/org/apache/solr/cluster/placement/NodeMetric.java
index 8f5781d1bd9..7ab6592e755 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/NodeMetric.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/NodeMetric.java
@@ -43,4 +43,20 @@ public interface NodeMetric<T> extends Metric<T> {
      */
     UNSPECIFIED
   }
+
+  /**
+   * Increase the metric by the given amount. This will be used while computing placements.
+   *
+   * @param a the original metric value
+   * @param b the amount to increase the metric by
+   */
+  T increase(T a, T b);
+
+  /**
+   * Decrease the metric by the given amount. This will be used while computing placements.
+   *
+   * @param a the original metric value
+   * @param b the amount to decrease the metric by
+   */
+  T decrease(T a, T b);
 }
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 251dd49759d..2cb20a9c827 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
@@ -33,6 +33,7 @@ import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.AttributeFetcher;
 import org.apache.solr.cluster.placement.AttributeValues;
 import org.apache.solr.cluster.placement.CollectionMetrics;
+import org.apache.solr.cluster.placement.Metric;
 import org.apache.solr.cluster.placement.NodeMetric;
 import org.apache.solr.cluster.placement.ReplicaMetric;
 import org.apache.solr.common.SolrException;
@@ -98,12 +99,12 @@ public class AttributeFetcherImpl implements AttributeFetcher {
     Map<Node, Set<String>> nodeToReplicaInternalTags = new HashMap<>();
     Map<String, Set<ReplicaMetric<?>>> requestedCollectionNamesMetrics =
         requestedCollectionMetrics.entrySet().stream()
-            .collect(Collectors.toMap(e -> e.getKey().getName(), e -> e.getValue()));
+            .collect(Collectors.toMap(e -> e.getKey().getName(), Map.Entry::getValue));
 
     // 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
+    // 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 = new HashMap<>();
     for (String sysPropSnitch : requestedNodeSystemSnitchTags) {
@@ -122,7 +123,7 @@ public class AttributeFetcherImpl implements AttributeFetcher {
     requestedCollectionMetrics.forEach(
         (collection, tags) -> {
           Set<String> collectionTags =
-              tags.stream().map(tag -> tag.getInternalName()).collect(Collectors.toSet());
+              tags.stream().map(Metric::getInternalName).collect(Collectors.toSet());
           collection
               .shards()
               .forEach(
@@ -179,8 +180,7 @@ public class AttributeFetcherImpl implements AttributeFetcher {
                               collectionMetricsBuilder
                                   .getShardMetricsBuilders()
                                   .computeIfAbsent(
-                                      shardName,
-                                      s -> new CollectionMetricsBuilder.ShardMetricsBuilder(s));
+                                      shardName, CollectionMetricsBuilder.ShardMetricsBuilder::new);
                           replicas.forEach(
                               replica -> {
                                 CollectionMetricsBuilder.ReplicaMetricsBuilder
@@ -189,9 +189,8 @@ public class AttributeFetcherImpl implements AttributeFetcher {
                                             .getReplicaMetricsBuilders()
                                             .computeIfAbsent(
                                                 replica.getName(),
-                                                n ->
-                                                    new CollectionMetricsBuilder
-                                                        .ReplicaMetricsBuilder(n));
+                                                CollectionMetricsBuilder.ReplicaMetricsBuilder
+                                                    ::new);
                                 replicaMetricsBuilder.setLeader(replica.isLeader());
                                 if (replica.isLeader()) {
                                   shardMetricsBuilder.setLeaderMetrics(replicaMetricsBuilder);
@@ -199,10 +198,9 @@ public class AttributeFetcherImpl implements AttributeFetcher {
                                 Set<ReplicaMetric<?>> requestedMetrics =
                                     requestedCollectionNamesMetrics.get(replica.getCollection());
                                 requestedMetrics.forEach(
-                                    metric -> {
-                                      replicaMetricsBuilder.addMetric(
-                                          metric, replica.get(metric.getInternalName()));
-                                    });
+                                    metric ->
+                                        replicaMetricsBuilder.addMetric(
+                                            metric, replica.get(metric.getInternalName())));
                               });
                         });
               });
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 58ab2efd98a..c69c4dcebb3 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
@@ -62,6 +62,28 @@ public class AttributeValuesImpl implements AttributeValues {
     return Optional.ofNullable((T) nodeToValue.get(node));
   }
 
+  @Override
+  @SuppressWarnings("unchecked")
+  public <T> boolean increaseNodeMetric(Node node, NodeMetric<T> metric, T byValue) {
+    Map<Node, Object> nodeToValue = metricSnitchToNodeToValue.get(metric);
+    if (nodeToValue == null) {
+      return false;
+    }
+    nodeToValue.merge(node, byValue, (orig, add) -> metric.increase((T) orig, (T) add));
+    return true;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public <T> boolean decreaseNodeMetric(Node node, NodeMetric<T> metric, T byValue) {
+    Map<Node, Object> nodeToValue = metricSnitchToNodeToValue.get(metric);
+    if (nodeToValue == null) {
+      return false;
+    }
+    nodeToValue.merge(node, byValue, (orig, add) -> metric.decrease((T) orig, (T) add));
+    return true;
+  }
+
   @Override
   public Optional<CollectionMetrics> getCollectionMetrics(String collectionName) {
     return Optional.ofNullable(collectionMetrics.get(collectionName));
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/BuiltInMetrics.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/BuiltInMetrics.java
new file mode 100644
index 00000000000..a1a59c09734
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/BuiltInMetrics.java
@@ -0,0 +1,79 @@
+/*
+ * 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.impl.NodeValueFetcher;
+import org.apache.solr.cluster.placement.NodeMetric;
+
+/** Node and Replica Metrics available to the placement plugins */
+public class BuiltInMetrics {
+
+  /*
+   * Node Metrics
+   */
+
+  /** Total disk space in GB. */
+  public static final NodeMetricImpl<Double> NODE_TOTAL_DISK_GB =
+      new NodeMetricImpl.DoubleNodeMetricImpl(
+          "totalDisk",
+          NodeMetric.Registry.SOLR_NODE,
+          "CONTAINER.fs.totalSpace",
+          MetricImpl.BYTES_TO_GB_CONVERTER);
+
+  /** Free (usable) disk space in GB. */
+  public static final NodeMetricImpl<Double> NODE_FREE_DISK_GB =
+      new NodeMetricImpl.DoubleNodeMetricImpl(
+          "freeDisk",
+          NodeMetric.Registry.SOLR_NODE,
+          "CONTAINER.fs.usableSpace",
+          MetricImpl.BYTES_TO_GB_CONVERTER);
+
+  /** Number of all cores. */
+  public static final NodeMetricImpl<Integer> NODE_NUM_CORES =
+      new NodeMetricImpl.IntNodeMetricImpl(NodeValueFetcher.CORES);
+
+  public static final NodeMetricImpl<Double> NODE_HEAP_USAGE =
+      new NodeMetricImpl.DoubleNodeMetricImpl(NodeValueFetcher.Tags.HEAPUSAGE.tagName);
+
+  /** System load average. */
+  public static final NodeMetricImpl<Double> NODE_SYSLOAD_AVG =
+      new NodeMetricImpl.DoubleNodeMetricImpl(
+          NodeValueFetcher.Tags.SYSLOADAVG.tagName,
+          NodeMetric.Registry.SOLR_JVM,
+          NodeValueFetcher.Tags.SYSLOADAVG.prefix);
+
+  /** Number of available processors. */
+  public static final NodeMetricImpl<Integer> NODE_AVAILABLE_PROCESSORS =
+      new NodeMetricImpl.IntNodeMetricImpl(
+          "availableProcessors", NodeMetric.Registry.SOLR_JVM, "os.availableProcessors");
+
+  /*
+   * Replica Metrics
+   */
+
+  /** Replica index size in GB. */
+  public static final ReplicaMetricImpl<Double> REPLICA_INDEX_SIZE_GB =
+      new ReplicaMetricImpl<>("sizeGB", "INDEX.sizeInBytes", MetricImpl.BYTES_TO_GB_CONVERTER);
+
+  /** 1-min query rate of the /select handler. */
+  public static final ReplicaMetricImpl<Double> REPLICA_QUERY_RATE_1MIN =
+      new ReplicaMetricImpl<>("queryRate", "QUERY./select.requestTimes:1minRate");
+  /** 1-min update rate of the /update handler. */
+  public static final ReplicaMetricImpl<Double> REPLICA_UPDATE_RATE_1MIN =
+      new ReplicaMetricImpl<>("updateRate", "UPDATE./update.requestTimes:1minRate");
+}
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/NodeMetricImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/NodeMetricImpl.java
index b76013d1903..329ef417de6 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/NodeMetricImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/NodeMetricImpl.java
@@ -19,42 +19,13 @@ package org.apache.solr.cluster.placement.impl;
 
 import java.util.Objects;
 import java.util.function.Function;
-import org.apache.solr.client.solrj.impl.NodeValueFetcher;
 import org.apache.solr.cluster.placement.NodeMetric;
 
 /**
  * Node metric identifier, corresponding to a node-level metric registry and the internal metric
  * name.
  */
-public class NodeMetricImpl<T> extends MetricImpl<T> implements NodeMetric<T> {
-
-  /** Total disk space in GB. */
-  public static final NodeMetricImpl<Double> TOTAL_DISK_GB =
-      new NodeMetricImpl<>(
-          "totalDisk", Registry.SOLR_NODE, "CONTAINER.fs.totalSpace", BYTES_TO_GB_CONVERTER);
-
-  /** Free (usable) disk space in GB. */
-  public static final NodeMetricImpl<Double> FREE_DISK_GB =
-      new NodeMetricImpl<>(
-          "freeDisk", Registry.SOLR_NODE, "CONTAINER.fs.usableSpace", BYTES_TO_GB_CONVERTER);
-
-  /** Number of all cores. */
-  public static final NodeMetricImpl<Integer> NUM_CORES =
-      new NodeMetricImpl<>(NodeValueFetcher.CORES);
-
-  public static final NodeMetricImpl<Double> HEAP_USAGE =
-      new NodeMetricImpl<>(NodeValueFetcher.Tags.HEAPUSAGE.tagName);
-
-  /** System load average. */
-  public static final NodeMetricImpl<Double> SYSLOAD_AVG =
-      new NodeMetricImpl<>(
-          NodeValueFetcher.Tags.SYSLOADAVG.tagName,
-          Registry.SOLR_JVM,
-          NodeValueFetcher.Tags.SYSLOADAVG.prefix);
-
-  /** Number of available processors. */
-  public static final NodeMetricImpl<Integer> AVAILABLE_PROCESSORS =
-      new NodeMetricImpl<>("availableProcessors", Registry.SOLR_JVM, "os.availableProcessors");
+public abstract class NodeMetricImpl<T> extends MetricImpl<T> implements NodeMetric<T> {
 
   private final Registry registry;
 
@@ -122,4 +93,118 @@ public class NodeMetricImpl<T> extends MetricImpl<T> implements NodeMetric<T> {
       return "NodeMetricImpl{key=" + internalName + "}";
     }
   }
+
+  static class IntNodeMetricImpl extends NodeMetricImpl<Integer> {
+
+    public IntNodeMetricImpl(String name, Registry registry, String internalName) {
+      super(name, registry, internalName);
+    }
+
+    public IntNodeMetricImpl(
+        String name, Registry registry, String internalName, Function<Object, Integer> converter) {
+      super(name, registry, internalName, converter);
+    }
+
+    public IntNodeMetricImpl(String key) {
+      super(key);
+    }
+
+    public IntNodeMetricImpl(String key, Function<Object, Integer> converter) {
+      super(key, converter);
+    }
+
+    @Override
+    public Integer increase(Integer a, Integer b) {
+      if (b == null) {
+        return a;
+      } else if (a == null) {
+        return b;
+      } else {
+        return a + b;
+      }
+    }
+
+    @Override
+    public Integer decrease(Integer a, Integer b) {
+      if (b == null) {
+        return a;
+      } else if (a == null) {
+        return b * -1;
+      } else {
+        return a - b;
+      }
+    }
+  }
+
+  static class DoubleNodeMetricImpl extends NodeMetricImpl<Double> {
+
+    public DoubleNodeMetricImpl(String name, Registry registry, String internalName) {
+      super(name, registry, internalName);
+    }
+
+    public DoubleNodeMetricImpl(
+        String name, Registry registry, String internalName, Function<Object, Double> converter) {
+      super(name, registry, internalName, converter);
+    }
+
+    public DoubleNodeMetricImpl(String key) {
+      super(key);
+    }
+
+    public DoubleNodeMetricImpl(String key, Function<Object, Double> converter) {
+      super(key, converter);
+    }
+
+    @Override
+    public Double increase(Double a, Double b) {
+      if (b == null) {
+        return a;
+      } else if (a == null) {
+        return b;
+      } else {
+        return a + b;
+      }
+    }
+
+    @Override
+    public Double decrease(Double a, Double b) {
+      if (b == null) {
+        return a;
+      } else if (a == null) {
+        return b * -1;
+      } else {
+        return a - b;
+      }
+    }
+  }
+
+  static class StaticNodeMetricImpl<T> extends NodeMetricImpl<T> {
+
+    public StaticNodeMetricImpl(String name, Registry registry, String internalName) {
+      super(name, registry, internalName);
+    }
+
+    public StaticNodeMetricImpl(
+        String name, Registry registry, String internalName, Function<Object, T> converter) {
+      super(name, registry, internalName, converter);
+    }
+
+    public StaticNodeMetricImpl(String key) {
+      super(key);
+    }
+
+    public StaticNodeMetricImpl(String key, Function<Object, T> converter) {
+      super(key, converter);
+    }
+
+    @Override
+    public T increase(T a, T b) {
+      return a;
+    }
+
+    @Override
+    public T decrease(T a, T b) {
+      return a;
+    }
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/ReplicaMetricImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/ReplicaMetricImpl.java
index 5deaf97d027..3e775339bb4 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/ReplicaMetricImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/ReplicaMetricImpl.java
@@ -25,17 +25,6 @@ import org.apache.solr.cluster.placement.ReplicaMetric;
  */
 public class ReplicaMetricImpl<T> extends MetricImpl<T> implements ReplicaMetric<T> {
 
-  /** Replica index size in GB. */
-  public static final ReplicaMetricImpl<Double> INDEX_SIZE_GB =
-      new ReplicaMetricImpl<>("sizeGB", "INDEX.sizeInBytes", BYTES_TO_GB_CONVERTER);
-
-  /** 1-min query rate of the /select handler. */
-  public static final ReplicaMetricImpl<Double> QUERY_RATE_1MIN =
-      new ReplicaMetricImpl<>("queryRate", "QUERY./select.requestTimes:1minRate");
-  /** 1-min update rate of the /update handler. */
-  public static final ReplicaMetricImpl<Double> UPDATE_RATE_1MIN =
-      new ReplicaMetricImpl<>("updateRate", "UPDATE./update.requestTimes:1minRate");
-
   public ReplicaMetricImpl(String name, String internalName) {
     super(name, internalName);
   }
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 1ec0de525c7..d5802fefa7a 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
@@ -58,8 +58,11 @@ import org.apache.solr.cluster.placement.PlacementPlanFactory;
 import org.apache.solr.cluster.placement.PlacementPlugin;
 import org.apache.solr.cluster.placement.PlacementPluginFactory;
 import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.ReplicaMetric;
+import org.apache.solr.cluster.placement.ReplicaMetrics;
 import org.apache.solr.cluster.placement.ReplicaPlacement;
-import org.apache.solr.cluster.placement.impl.NodeMetricImpl;
+import org.apache.solr.cluster.placement.ShardMetrics;
+import org.apache.solr.cluster.placement.impl.BuiltInMetrics;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.slf4j.Logger;
@@ -262,8 +265,10 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
         throws PlacementException {
       List<PlacementPlan> placementPlans = new ArrayList<>(requests.size());
       Set<Node> allNodes = new HashSet<>();
+      Set<SolrCollection> allCollections = new HashSet<>();
       for (PlacementRequest request : requests) {
         allNodes.addAll(request.getTargetNodes());
+        allCollections.add(request.getCollection());
       }
 
       // Fetch attributes for a superset of all nodes requested amongst the placementRequests
@@ -274,8 +279,12 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
           .requestNodeSystemProperty(AffinityPlacementConfig.REPLICA_TYPE_SYSPROP)
           .requestNodeSystemProperty(AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP);
       attributeFetcher
-          .requestNodeMetric(NodeMetricImpl.NUM_CORES)
-          .requestNodeMetric(NodeMetricImpl.FREE_DISK_GB);
+          .requestNodeMetric(BuiltInMetrics.NODE_NUM_CORES)
+          .requestNodeMetric(BuiltInMetrics.NODE_FREE_DISK_GB);
+      Set<ReplicaMetric<?>> replicaMetrics = Set.of(BuiltInMetrics.REPLICA_INDEX_SIZE_GB);
+      for (SolrCollection collection : allCollections) {
+        attributeFetcher.requestCollectionMetrics(collection, replicaMetrics);
+      }
       attributeFetcher.fetchFrom(allNodes);
       final AttributeValues attrValues = attributeFetcher.fetchAttributes();
       // Get the number of currently existing cores per node, so we can update as we place new cores
@@ -296,12 +305,6 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
         // filter out nodes that don't match the "node types" specified in the collection props
         nodes = filterNodesByNodeType(placementContext.getCluster(), request, attrValues, nodes);
 
-        // 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.
-        EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes =
-            getAvailableNodesForReplicaTypes(nodes, attrValues);
-
         // 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
@@ -314,6 +317,24 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
         // 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()) {
+          ReplicaMetrics leaderMetrics =
+              attrValues
+                  .getCollectionMetrics(solrCollection.getName())
+                  .flatMap(colMetrics -> colMetrics.getShardMetrics(shardName))
+                  .flatMap(ShardMetrics::getLeaderMetrics)
+                  .orElse(null);
+
+          // 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.
+          //
+          // This also filters out nodes that will not satisfy the rules if the replica is placed
+          // there
+          EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes =
+              getAvailableNodesForReplicaTypes(nodes, attrValues, leaderMetrics);
+
           // Inventory nodes (if any) that already have a replica of any type for the shard, because
           // we can't be placing additional replicas on these. This data structure is updated after
           // each replica to node assign and is used to make sure different replica types are not
@@ -342,19 +363,23 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
           // 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,
-                nodesWithReplicas,
-                allCoresOnNodes,
-                placementContext.getPlacementPlanFactory(),
-                replicaPlacements,
-                doSpreadAcrossDomains);
+            int numReplicasToCreate = request.getCountReplicasToCreate(replicaType);
+            if (numReplicasToCreate > 0) {
+              makePlacementDecisions(
+                  solrCollection,
+                  shardName,
+                  availabilityZones,
+                  replicaType,
+                  numReplicasToCreate,
+                  attrValues,
+                  leaderMetrics,
+                  replicaTypeToNodes,
+                  nodesWithReplicas,
+                  allCoresOnNodes,
+                  placementContext.getPlacementPlanFactory(),
+                  replicaPlacements,
+                  doSpreadAcrossDomains);
+            }
           }
         }
         placementPlans.add(
@@ -390,7 +415,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
     @Override
     public void verifyAllowedModification(
         ModificationRequest modificationRequest, PlacementContext placementContext)
-        throws PlacementModificationException, InterruptedException {
+        throws PlacementModificationException {
       if (modificationRequest instanceof DeleteShardsRequest) {
         log.warn("DeleteShardsRequest not implemented yet, skipping: {}", modificationRequest);
       } else if (modificationRequest instanceof DeleteCollectionRequest) {
@@ -404,7 +429,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
 
     private void verifyDeleteCollection(
         DeleteCollectionRequest deleteCollectionRequest, PlacementContext placementContext)
-        throws PlacementModificationException, InterruptedException {
+        throws PlacementModificationException {
       Cluster cluster = placementContext.getCluster();
       Set<String> colocatedCollections =
           colocatedWith.getOrDefault(deleteCollectionRequest.getCollection().getName(), Set.of());
@@ -428,7 +453,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
 
     private void verifyDeleteReplicas(
         DeleteReplicasRequest deleteReplicasRequest, PlacementContext placementContext)
-        throws PlacementModificationException, InterruptedException {
+        throws PlacementModificationException {
       Cluster cluster = placementContext.getCluster();
       SolrCollection secondaryCollection = deleteReplicasRequest.getCollection();
       Set<String> colocatedCollections = colocatedWith.get(secondaryCollection.getName());
@@ -443,13 +468,12 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
                   shard
                       .replicas()
                       .forEach(
-                          replica -> {
-                            secondaryNodeShardReplicas
-                                .computeIfAbsent(replica.getNode(), n -> new HashMap<>())
-                                .computeIfAbsent(
-                                    replica.getShard().getShardName(), s -> new AtomicInteger())
-                                .incrementAndGet();
-                          }));
+                          replica ->
+                              secondaryNodeShardReplicas
+                                  .computeIfAbsent(replica.getNode(), n -> new HashMap<>())
+                                  .computeIfAbsent(
+                                      replica.getShard().getShardName(), s -> new AtomicInteger())
+                                  .incrementAndGet()));
 
       // find the colocated-with collections
       Map<Node, Set<String>> colocatingNodes = new HashMap<>();
@@ -462,11 +486,10 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
                       shard
                           .replicas()
                           .forEach(
-                              replica -> {
-                                colocatingNodes
-                                    .computeIfAbsent(replica.getNode(), n -> new HashSet<>())
-                                    .add(coll.getName());
-                              }));
+                              replica ->
+                                  colocatingNodes
+                                      .computeIfAbsent(replica.getNode(), n -> new HashSet<>())
+                                      .add(coll.getName())));
         }
       } catch (IOException ioe) {
         throw new PlacementModificationException(
@@ -540,6 +563,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
       private final Random random;
       private final List<Node> availableNodesForPlacement;
       private final AttributeValues attributeValues;
+      private final ReplicaMetrics leaderMetrics;
       private TreeSet<SpreadDomainWithNodes> sortedSpreadDomains;
       private final Map<String, Integer> currentSpreadDomainUsageUsage;
       private int numNodesForPlacement;
@@ -551,6 +575,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
           Comparator<Node> nodeComparator,
           Random random,
           AttributeValues attributeValues,
+          ReplicaMetrics leaderMetrics,
           Map<String, Integer> currentSpreadDomainUsageUsage) {
         this.azName = azName;
         this.availableNodesForPlacement = availableNodesForPlacement;
@@ -558,6 +583,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
         this.nodeComparator = nodeComparator;
         this.random = random;
         this.attributeValues = attributeValues;
+        this.leaderMetrics = leaderMetrics;
         this.currentSpreadDomainUsageUsage = currentSpreadDomainUsageUsage;
         this.numNodesForPlacement = availableNodesForPlacement.size();
       }
@@ -640,19 +666,27 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
       public Node removeBestNode() {
         assert hasBeenSorted();
         this.numNodesForPlacement--;
+        Node n;
         if (useSpreadDomains) {
           // Since this SpreadDomainWithNodes needs to be re-sorted in the sortedSpreadDomains, we
           // remove it and then re-add it, once the best node has been removed.
           SpreadDomainWithNodes group = sortedSpreadDomains.pollFirst();
-          Node n = group.sortedNodesForPlacement.remove(0);
+          n = group.sortedNodesForPlacement.remove(0);
           this.currentSpreadDomainUsageUsage.merge(group.spreadDomainName, 1, Integer::sum);
           if (!group.sortedNodesForPlacement.isEmpty()) {
             sortedSpreadDomains.add(group);
           }
-          return n;
         } else {
-          return availableNodesForPlacement.remove(0);
+          n = availableNodesForPlacement.remove(0);
         }
+        Optional.ofNullable(leaderMetrics)
+            .flatMap(lrm -> lrm.getReplicaMetric(BuiltInMetrics.REPLICA_INDEX_SIZE_GB))
+            .ifPresent(
+                indexSize ->
+                    attributeValues.decreaseNodeMetric(
+                        n, BuiltInMetrics.NODE_FREE_DISK_GB, indexSize));
+        attributeValues.increaseNodeMetric(n, BuiltInMetrics.NODE_NUM_CORES, 1);
+        return n;
       }
 
       public int numNodes() {
@@ -734,7 +768,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
 
       for (Node node : nodes) {
         attrValues
-            .getNodeMetric(node, NodeMetricImpl.NUM_CORES)
+            .getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES)
             .ifPresent(count -> coresOnNodes.put(node, count));
       }
 
@@ -754,7 +788,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
      *     node.
      */
     private EnumMap<Replica.ReplicaType, Set<Node>> getAvailableNodesForReplicaTypes(
-        Set<Node> nodes, final AttributeValues attrValues) {
+        Set<Node> nodes, final AttributeValues attrValues, final ReplicaMetrics leaderMetrics) {
       EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes =
           new EnumMap<>(Replica.ReplicaType.class);
 
@@ -764,7 +798,9 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
 
       for (Node node : nodes) {
         // Exclude nodes with unknown or too small disk free space
-        if (attrValues.getNodeMetric(node, NodeMetricImpl.FREE_DISK_GB).isEmpty()) {
+        Optional<Double> nodeFreeDiskGB =
+            attrValues.getNodeMetric(node, BuiltInMetrics.NODE_FREE_DISK_GB);
+        if (nodeFreeDiskGB.isEmpty()) {
           if (log.isWarnEnabled()) {
             log.warn(
                 "Unknown free disk on node {}, excluding it from placement decisions.",
@@ -774,18 +810,26 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
           // CoresAndDiskComparator), be careful it you change anything here.
           continue;
         }
-        if (attrValues.getNodeMetric(node, NodeMetricImpl.FREE_DISK_GB).get() < minimalFreeDiskGB) {
+        double replicaIndexSize =
+            Optional.ofNullable(leaderMetrics)
+                .flatMap(lm -> lm.getReplicaMetric(BuiltInMetrics.REPLICA_INDEX_SIZE_GB))
+                .orElse(0D);
+        double projectedFreeDiskIfPlaced =
+            BuiltInMetrics.NODE_FREE_DISK_GB.decrease(nodeFreeDiskGB.get(), replicaIndexSize);
+        if (projectedFreeDiskIfPlaced < minimalFreeDiskGB) {
           if (log.isWarnEnabled()) {
             log.warn(
-                "Node {} free disk ({}GB) lower than configured minimum {}GB, excluding it from placement decisions.",
+                "Node {} free disk ({}GB) minus the projected replica size ({}GB) is lower than configured"
+                    + " minimum {}GB, excluding it from placement decisions.",
                 node.getName(),
-                attrValues.getNodeMetric(node, NodeMetricImpl.FREE_DISK_GB).get(),
+                nodeFreeDiskGB.get(),
+                replicaIndexSize,
                 minimalFreeDiskGB);
           }
           continue;
         }
 
-        if (attrValues.getNodeMetric(node, NodeMetricImpl.NUM_CORES).isEmpty()) {
+        if (attrValues.getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES).isEmpty()) {
           if (log.isWarnEnabled()) {
             log.warn(
                 "Unknown number of cores on node {}, excluding it from placement decisions.",
@@ -839,9 +883,9 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
      *   <li>Place replicas if 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 #getAvailableNodesForReplicaTypes(Set,
-     *       AttributeValues)}). 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.
+     *       AttributeValues, ReplicaMetrics)}). 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>Place replicas on nodes having a smaller number of cores (the number of cores
      *       considered for this decision includes previous placement decisions made during the
      *       processing of the placement request)
@@ -857,6 +901,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
         Replica.ReplicaType replicaType,
         int numReplicas,
         final AttributeValues attrValues,
+        final ReplicaMetrics leaderMetrics,
         EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes,
         Set<Node> nodesWithReplicas,
         Map<Node, Integer> coresOnNodes,
@@ -954,6 +999,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
                     interGroupNodeComparator,
                     replicaPlacementRandom,
                     attrValues,
+                    leaderMetrics,
                     spreadDomainsInUse));
       }
 
@@ -1124,7 +1170,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
                     Optional<String> nodePropOpt =
                         attributeValues.getSystemProperty(
                             n, AffinityPlacementConfig.NODE_TYPE_SYSPROP);
-                    if (!nodePropOpt.isPresent()) {
+                    if (nodePropOpt.isEmpty()) {
                       return false;
                     }
                     Set<String> nodeTypes =
@@ -1172,9 +1218,11 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
       public int compare(Node a, Node b) {
         // Note all nodes do have free disk defined. This has been verified earlier.
         boolean aHasLowFreeSpace =
-            attrValues.getNodeMetric(a, NodeMetricImpl.FREE_DISK_GB).get() < prioritizedFreeDiskGB;
+            attrValues.getNodeMetric(a, BuiltInMetrics.NODE_FREE_DISK_GB).orElse(0D)
+                < prioritizedFreeDiskGB;
         boolean bHasLowFreeSpace =
-            attrValues.getNodeMetric(b, NodeMetricImpl.FREE_DISK_GB).get() < prioritizedFreeDiskGB;
+            attrValues.getNodeMetric(b, BuiltInMetrics.NODE_FREE_DISK_GB).orElse(0D)
+                < 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
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 298c0d3687e..3f0fa3d86d9 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,7 +40,7 @@ import org.apache.solr.cluster.placement.PlacementPlugin;
 import org.apache.solr.cluster.placement.PlacementPluginFactory;
 import org.apache.solr.cluster.placement.PlacementRequest;
 import org.apache.solr.cluster.placement.ReplicaPlacement;
-import org.apache.solr.cluster.placement.impl.NodeMetricImpl;
+import org.apache.solr.cluster.placement.impl.BuiltInMetrics;
 import org.apache.solr.common.util.SuppressForbidden;
 
 /**
@@ -76,16 +76,16 @@ public class MinimizeCoresPlacementFactory
 
       // Fetch attributes for a superset of all nodes requested amongst the placementRequests
       AttributeFetcher attributeFetcher = placementContext.getAttributeFetcher();
-      attributeFetcher.requestNodeMetric(NodeMetricImpl.NUM_CORES);
+      attributeFetcher.requestNodeMetric(BuiltInMetrics.NODE_NUM_CORES);
       attributeFetcher.fetchFrom(allNodes);
       AttributeValues attrValues = attributeFetcher.fetchAttributes();
       Map<String, Integer> coresPerNodeTotal = new HashMap<>();
       for (Node node : allNodes) {
-        if (attrValues.getNodeMetric(node, NodeMetricImpl.NUM_CORES).isEmpty()) {
+        if (attrValues.getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES).isEmpty()) {
           throw new PlacementException("Can't get number of cores in " + node);
         }
         coresPerNodeTotal.put(
-            node.getName(), attrValues.getNodeMetric(node, NodeMetricImpl.NUM_CORES).get());
+            node.getName(), attrValues.getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES).get());
       }
 
       for (PlacementRequest request : requests) {
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 fb7d769d1c9..2ef8ed84b29 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
@@ -404,48 +404,52 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
     Cluster cluster = new SimpleClusterAbstractionsImpl.ClusterImpl(cloudManager);
     SolrCollection collection = cluster.getCollection(COLLECTION);
     AttributeFetcher attributeFetcher = new AttributeFetcherImpl(cloudManager);
-    NodeMetric<String> someMetricKey = new NodeMetricImpl<>("solr.jvm:system.properties:user.name");
+    NodeMetric<String> someMetricKey =
+        new NodeMetricImpl.StaticNodeMetricImpl<>("solr.jvm:system.properties:user.name");
     String sysprop = "user.name";
     attributeFetcher
         .fetchFrom(cluster.getLiveNodes())
-        .requestNodeMetric(NodeMetricImpl.HEAP_USAGE)
-        .requestNodeMetric(NodeMetricImpl.SYSLOAD_AVG)
-        .requestNodeMetric(NodeMetricImpl.NUM_CORES)
-        .requestNodeMetric(NodeMetricImpl.FREE_DISK_GB)
-        .requestNodeMetric(NodeMetricImpl.TOTAL_DISK_GB)
-        .requestNodeMetric(NodeMetricImpl.AVAILABLE_PROCESSORS)
+        .requestNodeMetric(BuiltInMetrics.NODE_HEAP_USAGE)
+        .requestNodeMetric(BuiltInMetrics.NODE_SYSLOAD_AVG)
+        .requestNodeMetric(BuiltInMetrics.NODE_NUM_CORES)
+        .requestNodeMetric(BuiltInMetrics.NODE_FREE_DISK_GB)
+        .requestNodeMetric(BuiltInMetrics.NODE_TOTAL_DISK_GB)
+        .requestNodeMetric(BuiltInMetrics.NODE_AVAILABLE_PROCESSORS)
         .requestNodeMetric(someMetricKey)
         .requestNodeSystemProperty(sysprop)
         .requestCollectionMetrics(
             collection,
             Set.of(
-                ReplicaMetricImpl.INDEX_SIZE_GB,
-                ReplicaMetricImpl.QUERY_RATE_1MIN,
-                ReplicaMetricImpl.UPDATE_RATE_1MIN));
+                BuiltInMetrics.REPLICA_INDEX_SIZE_GB,
+                BuiltInMetrics.REPLICA_QUERY_RATE_1MIN,
+                BuiltInMetrics.REPLICA_UPDATE_RATE_1MIN));
     AttributeValues attributeValues = attributeFetcher.fetchAttributes();
     String userName = System.getProperty("user.name");
     // node metrics
     for (Node node : cluster.getLiveNodes()) {
-      Optional<Double> doubleOpt = attributeValues.getNodeMetric(node, NodeMetricImpl.HEAP_USAGE);
+      Optional<Double> doubleOpt =
+          attributeValues.getNodeMetric(node, BuiltInMetrics.NODE_HEAP_USAGE);
       assertTrue("heap usage", doubleOpt.isPresent());
       assertTrue(
           "heap usage should be 0 < heapUsage < 100 but was " + doubleOpt,
           doubleOpt.get() > 0 && doubleOpt.get() < 100);
-      doubleOpt = attributeValues.getNodeMetric(node, NodeMetricImpl.TOTAL_DISK_GB);
+      doubleOpt = attributeValues.getNodeMetric(node, BuiltInMetrics.NODE_TOTAL_DISK_GB);
       assertTrue("total disk", doubleOpt.isPresent());
       assertTrue("total disk should be > 0 but was " + doubleOpt, doubleOpt.get() > 0);
-      doubleOpt = attributeValues.getNodeMetric(node, NodeMetricImpl.FREE_DISK_GB);
+      doubleOpt = attributeValues.getNodeMetric(node, BuiltInMetrics.NODE_FREE_DISK_GB);
       assertTrue("free disk", doubleOpt.isPresent());
       assertTrue("free disk should be > 0 but was " + doubleOpt, doubleOpt.get() > 0);
-      Optional<Integer> intOpt = attributeValues.getNodeMetric(node, NodeMetricImpl.NUM_CORES);
+      Optional<Integer> intOpt = attributeValues.getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES);
       assertTrue("cores", intOpt.isPresent());
       assertTrue("cores should be > 0", intOpt.get() > 0);
       assertTrue(
           "systemLoadAverage 2",
-          attributeValues.getNodeMetric(node, NodeMetricImpl.SYSLOAD_AVG).isPresent());
+          attributeValues.getNodeMetric(node, BuiltInMetrics.NODE_SYSLOAD_AVG).isPresent());
       assertTrue(
           "availableProcessors",
-          attributeValues.getNodeMetric(node, NodeMetricImpl.AVAILABLE_PROCESSORS).isPresent());
+          attributeValues
+              .getNodeMetric(node, BuiltInMetrics.NODE_AVAILABLE_PROCESSORS)
+              .isPresent());
       Optional<String> userNameOpt = attributeValues.getNodeMetric(node, someMetricKey);
       assertTrue("user.name", userNameOpt.isPresent());
       assertEquals("userName", userName, userNameOpt.get());
@@ -471,7 +475,7 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
                         assertTrue("replica metrics", replicaMetricsOpt.isPresent());
                         ReplicaMetrics replicaMetrics = replicaMetricsOpt.get();
                         Optional<Double> indexSizeOpt =
-                            replicaMetrics.getReplicaMetric(ReplicaMetricImpl.INDEX_SIZE_GB);
+                            replicaMetrics.getReplicaMetric(BuiltInMetrics.REPLICA_INDEX_SIZE_GB);
                         assertTrue("indexSize", indexSizeOpt.isPresent());
                         indexSizeOpt.get();
                         assertTrue(
@@ -483,10 +487,12 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
 
                         assertNotNull(
                             "queryRate",
-                            replicaMetrics.getReplicaMetric(ReplicaMetricImpl.QUERY_RATE_1MIN));
+                            replicaMetrics.getReplicaMetric(
+                                BuiltInMetrics.REPLICA_QUERY_RATE_1MIN));
                         assertNotNull(
                             "updateRate",
-                            replicaMetrics.getReplicaMetric(ReplicaMetricImpl.UPDATE_RATE_1MIN));
+                            replicaMetrics.getReplicaMetric(
+                                BuiltInMetrics.REPLICA_UPDATE_RATE_1MIN));
                       });
             });
   }
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 0e1665116ac..66fee222e3d 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
@@ -19,6 +19,7 @@ package org.apache.solr.cluster.placement.plugins;
 
 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;
@@ -44,11 +45,13 @@ import org.apache.solr.cluster.placement.PlacementContext;
 import org.apache.solr.cluster.placement.PlacementException;
 import org.apache.solr.cluster.placement.PlacementPlan;
 import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementRequest;
 import org.apache.solr.cluster.placement.ReplicaPlacement;
 import org.apache.solr.cluster.placement.impl.ModificationRequestImpl;
 import org.apache.solr.cluster.placement.impl.PlacementRequestImpl;
 import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.StrUtils;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -811,6 +814,72 @@ public class AffinityPlacementFactoryTest extends SolrTestCaseJ4 {
     }
   }
 
+  @Test
+  public void testFreeDiskConstraintsWithNewReplicas() throws Exception {
+    String collectionName = "freeDiskWithReplicasCollection";
+    int NUM_NODES = 3;
+    Builders.ClusterBuilder clusterBuilder =
+        Builders.newClusterBuilder().initializeLiveNodes(NUM_NODES);
+    Node smallNode = null;
+    for (int i = 0; i < NUM_NODES; i++) {
+      Builders.NodeBuilder nodeBuilder = clusterBuilder.getLiveNodeBuilders().get(i);
+      // Act as if the two replicas were placed on nodes 1 and 2
+      nodeBuilder.setCoreCount(0);
+      nodeBuilder.setFreeDiskGB(100.0);
+    }
+
+    Builders.CollectionBuilder collectionBuilder = Builders.newCollectionBuilder(collectionName);
+    collectionBuilder.initializeShardsReplicas(
+        3,
+        1,
+        0,
+        0,
+        clusterBuilder.getLiveNodeBuilders(), // .subList(1, 3),
+        List.of(33, 33, 60));
+    clusterBuilder.addCollection(collectionBuilder);
+
+    PlacementContext placementContext = clusterBuilder.buildPlacementContext();
+    Cluster cluster = placementContext.getCluster();
+
+    SolrCollection solrCollection = cluster.getCollection(collectionName);
+
+    // Test when an additional replicaType makes the projected indexSize go over the limit
+    // Add two replicas (different types) to the first shard, the second replica should fail
+    PlacementRequestImpl badReplicaPlacementRequest =
+        new PlacementRequestImpl(
+            solrCollection,
+            StreamSupport.stream(solrCollection.shards().spliterator(), false)
+                .map(Shard::getShardName)
+                .findFirst()
+                .map(Set::of)
+                .orElseGet(Collections::emptySet),
+            cluster.getLiveNodes(),
+            0,
+            1,
+            1);
+
+    Assert.assertThrows(
+        PlacementException.class,
+        () -> plugin.computePlacement(badReplicaPlacementRequest, placementContext));
+
+    // Test when an additional shard makes the projected indexSize go over the limit
+    // Add one replica to each shard, the third shard should fail
+    PlacementRequest badShardPlacementRequest =
+        new PlacementRequestImpl(
+            solrCollection,
+            StreamSupport.stream(solrCollection.shards().spliterator(), false)
+                .map(Shard::getShardName)
+                .collect(Collectors.toSet()),
+            cluster.getLiveNodes(),
+            0,
+            1,
+            1);
+
+    Assert.assertThrows(
+        PlacementException.class,
+        () -> plugin.computePlacement(badShardPlacementRequest, placementContext));
+  }
+
   @Test
   public void testWithCollectionPlacement() throws Exception {
     AffinityPlacementConfig config =
@@ -1294,6 +1363,12 @@ public class AffinityPlacementFactoryTest extends SolrTestCaseJ4 {
 
     Builders.ClusterBuilder clusterBuilder = Builders.newClusterBuilder().initializeLiveNodes(3);
     List<Builders.NodeBuilder> nodeBuilders = clusterBuilder.getLiveNodeBuilders();
+    // The first node needs to have 2 fewer cores than the second node, because unfortunately the
+    // metrics will be altered when computing the first placementRequest. If the metrics were copied
+    // when fetched, then this wouldn't be necessary. However, for now this is acceptable, because
+    // it's only a testing issue. The real AttributeFetcher does not share maps across the
+    // AttributeValues that it creates. So a placementPlugin gets a clean set of metrics for each
+    // placementRequest, that the placementPlugin can edit however it wants to.
     nodeBuilders
         .get(0)
         .setCoreCount(1)
@@ -1301,12 +1376,12 @@ public class AffinityPlacementFactoryTest extends SolrTestCaseJ4 {
         .setSysprop(AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP, "A");
     nodeBuilders
         .get(1)
-        .setCoreCount(2)
+        .setCoreCount(3)
         .setFreeDiskGB((double) (PRIORITIZED_FREE_DISK_GB + 1))
         .setSysprop(AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP, "A");
     nodeBuilders
         .get(2)
-        .setCoreCount(3)
+        .setCoreCount(4)
         .setFreeDiskGB((double) (PRIORITIZED_FREE_DISK_GB + 1))
         .setSysprop(AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP, "B");
 
@@ -1314,12 +1389,12 @@ public class AffinityPlacementFactoryTest extends SolrTestCaseJ4 {
 
     if (hasExistingCollection) {
       // Existing collection has replicas for its shards and is visible in the cluster state
-      collectionBuilder.initializeShardsReplicas(1, 1, 0, 0, nodeBuilders);
+      collectionBuilder.initializeShardsReplicas(1, 1, 0, 0, nodeBuilders, List.of(0));
       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());
+      collectionBuilder.initializeShardsReplicas(1, 0, 0, 0, List.of(), List.of(0));
     }
 
     PlacementContext placementContext = clusterBuilder.buildPlacementContext();
diff --git a/solr/test-framework/src/java/org/apache/solr/cluster/placement/Builders.java b/solr/test-framework/src/java/org/apache/solr/cluster/placement/Builders.java
index 776f5ff2dd7..3c18850e112 100644
--- a/solr/test-framework/src/java/org/apache/solr/cluster/placement/Builders.java
+++ b/solr/test-framework/src/java/org/apache/solr/cluster/placement/Builders.java
@@ -31,8 +31,8 @@ import org.apache.solr.cluster.Shard;
 import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.impl.AttributeFetcherImpl;
 import org.apache.solr.cluster.placement.impl.AttributeValuesImpl;
+import org.apache.solr.cluster.placement.impl.BuiltInMetrics;
 import org.apache.solr.cluster.placement.impl.CollectionMetricsBuilder;
-import org.apache.solr.cluster.placement.impl.NodeMetricImpl;
 import org.apache.solr.cluster.placement.impl.PlacementPlanFactoryImpl;
 import org.apache.solr.cluster.placement.impl.ReplicaMetricImpl;
 import org.apache.solr.common.util.Pair;
@@ -56,7 +56,7 @@ public class Builders {
     /** {@link NodeBuilder} for the live nodes of the cluster. */
     private List<NodeBuilder> nodeBuilders = new ArrayList<>();
 
-    private final List<CollectionBuilder> collectionBuilders = new ArrayList<>();
+    private final Map<String, CollectionBuilder> collectionBuilders = new HashMap<>();
 
     public ClusterBuilder initializeLiveNodes(int countNodes) {
       nodeBuilders = new ArrayList<>();
@@ -76,7 +76,7 @@ public class Builders {
     }
 
     public ClusterBuilder addCollection(CollectionBuilder collectionBuilder) {
-      collectionBuilders.add(collectionBuilder);
+      collectionBuilders.put(collectionBuilder.collectionName, collectionBuilder);
       return this;
     }
 
@@ -97,7 +97,7 @@ public class Builders {
 
     Map<String, SolrCollection> buildClusterCollections() {
       Map<String, SolrCollection> clusterCollections = new LinkedHashMap<>();
-      for (CollectionBuilder collectionBuilder : collectionBuilders) {
+      for (CollectionBuilder collectionBuilder : collectionBuilders.values()) {
         SolrCollection solrCollection = collectionBuilder.build();
         clusterCollections.put(solrCollection.getName(), solrCollection);
       }
@@ -144,17 +144,17 @@ public class Builders {
 
         if (nodeBuilder.getCoreCount() != null) {
           metrics
-              .computeIfAbsent(NodeMetricImpl.NUM_CORES, n -> new HashMap<>())
+              .computeIfAbsent(BuiltInMetrics.NODE_NUM_CORES, n -> new HashMap<>())
               .put(node, nodeBuilder.getCoreCount());
         }
         if (nodeBuilder.getFreeDiskGB() != null) {
           metrics
-              .computeIfAbsent(NodeMetricImpl.FREE_DISK_GB, n -> new HashMap<>())
+              .computeIfAbsent(BuiltInMetrics.NODE_FREE_DISK_GB, n -> new HashMap<>())
               .put(node, nodeBuilder.getFreeDiskGB());
         }
         if (nodeBuilder.getTotalDiskGB() != null) {
           metrics
-              .computeIfAbsent(NodeMetricImpl.TOTAL_DISK_GB, n -> new HashMap<>())
+              .computeIfAbsent(BuiltInMetrics.NODE_TOTAL_DISK_GB, n -> new HashMap<>())
               .put(node, nodeBuilder.getTotalDiskGB());
         }
         if (nodeBuilder.getSysprops() != null) {
@@ -177,11 +177,13 @@ public class Builders {
 
       if (!collectionBuilders.isEmpty()) {
         Map<Node, Object> nodeToCoreCount =
-            metrics.computeIfAbsent(NodeMetricImpl.NUM_CORES, n -> new HashMap<>());
+            metrics.computeIfAbsent(BuiltInMetrics.NODE_NUM_CORES, n -> new HashMap<>());
+        Map<Node, Object> nodeToFreeDisk =
+            metrics.computeIfAbsent(BuiltInMetrics.NODE_FREE_DISK_GB, n -> new HashMap<>());
         collectionBuilders.forEach(
-            builder -> {
-              collectionMetrics.put(
-                  builder.collectionName, builder.collectionMetricsBuilder.build());
+            (collName, builder) -> {
+              CollectionMetrics thisCollMetrics = builder.collectionMetricsBuilder.build();
+              collectionMetrics.put(collName, thisCollMetrics);
               SolrCollection collection = builder.build();
               collection
                   .iterator()
@@ -195,6 +197,23 @@ public class Builders {
                                         replica.getNode(),
                                         (node, count) ->
                                             (count == null) ? 1 : ((Number) count).intValue() + 1);
+                                    nodeToFreeDisk.computeIfPresent(
+                                        replica.getNode(),
+                                        (node, freeDisk) ->
+                                            BuiltInMetrics.NODE_FREE_DISK_GB.decrease(
+                                                (Double) freeDisk,
+                                                thisCollMetrics
+                                                    .getShardMetrics(shard.getShardName())
+                                                    .flatMap(
+                                                        m ->
+                                                            m.getReplicaMetrics(
+                                                                replica.getReplicaName()))
+                                                    .flatMap(
+                                                        m ->
+                                                            m.getReplicaMetric(
+                                                                BuiltInMetrics
+                                                                    .REPLICA_INDEX_SIZE_GB))
+                                                    .orElse(0D)));
                                   }));
             });
       }
@@ -411,7 +430,7 @@ public class Builders {
             shardMetricsBuilder.getReplicaMetricsBuilders().put(replicaName, replicaMetricsBuilder);
             if (initialSizeGBPerShard != null) {
               replicaMetricsBuilder.addMetric(
-                  ReplicaMetricImpl.INDEX_SIZE_GB,
+                  BuiltInMetrics.REPLICA_INDEX_SIZE_GB,
                   initialSizeGBPerShard.get(shardNumber - 1) * ReplicaMetricImpl.GB);
             }
             if (leader == null && type != Replica.ReplicaType.PULL) {
diff --git a/solr/test-framework/src/test/org/apache/solr/cluster/placement/BuildersTest.java b/solr/test-framework/src/test/org/apache/solr/cluster/placement/BuildersTest.java
index c012c8e6454..a43c262d9df 100644
--- a/solr/test-framework/src/test/org/apache/solr/cluster/placement/BuildersTest.java
+++ b/solr/test-framework/src/test/org/apache/solr/cluster/placement/BuildersTest.java
@@ -29,8 +29,7 @@ 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.impl.NodeMetricImpl;
-import org.apache.solr.cluster.placement.impl.ReplicaMetricImpl;
+import org.apache.solr.cluster.placement.impl.BuiltInMetrics;
 import org.junit.Test;
 
 /** */
@@ -85,17 +84,19 @@ public class BuildersTest extends SolrTestCaseJ4 {
     AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
     attributeFetcher
         .fetchFrom(cluster.getLiveNodes())
-        .requestNodeMetric(NodeMetricImpl.NUM_CORES)
-        .requestNodeMetric(NodeMetricImpl.FREE_DISK_GB)
-        .requestNodeMetric(NodeMetricImpl.TOTAL_DISK_GB)
-        .requestCollectionMetrics(collection, Set.of(ReplicaMetricImpl.INDEX_SIZE_GB));
+        .requestNodeMetric(BuiltInMetrics.NODE_NUM_CORES)
+        .requestNodeMetric(BuiltInMetrics.NODE_FREE_DISK_GB)
+        .requestNodeMetric(BuiltInMetrics.NODE_TOTAL_DISK_GB)
+        .requestCollectionMetrics(collection, Set.of(BuiltInMetrics.REPLICA_INDEX_SIZE_GB));
     AttributeValues attributeValues = attributeFetcher.fetchAttributes();
     for (Node node : cluster.getLiveNodes()) {
-      Optional<Integer> coreCount = attributeValues.getNodeMetric(node, NodeMetricImpl.NUM_CORES);
+      Optional<Integer> coreCount =
+          attributeValues.getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES);
       assertTrue("coreCount present", coreCount.isPresent());
-      Optional<Double> diskOpt = attributeValues.getNodeMetric(node, NodeMetricImpl.FREE_DISK_GB);
+      Optional<Double> diskOpt =
+          attributeValues.getNodeMetric(node, BuiltInMetrics.NODE_FREE_DISK_GB);
       assertTrue("freeDisk", diskOpt.isPresent());
-      diskOpt = attributeValues.getNodeMetric(node, NodeMetricImpl.TOTAL_DISK_GB);
+      diskOpt = attributeValues.getNodeMetric(node, BuiltInMetrics.NODE_TOTAL_DISK_GB);
       assertTrue("totalDisk", diskOpt.isPresent());
     }
     Optional<CollectionMetrics> collectionMetricsOpt =
@@ -109,7 +110,8 @@ public class BuildersTest extends SolrTestCaseJ4 {
       Optional<ReplicaMetrics> replicaMetricsOpt = shardMetrics.getLeaderMetrics();
       assertTrue("leader metrics", replicaMetricsOpt.isPresent());
       ReplicaMetrics leaderMetrics = replicaMetricsOpt.get();
-      Optional<Double> sizeOpt = leaderMetrics.getReplicaMetric(ReplicaMetricImpl.INDEX_SIZE_GB);
+      Optional<Double> sizeOpt =
+          leaderMetrics.getReplicaMetric(BuiltInMetrics.REPLICA_INDEX_SIZE_GB);
       assertTrue("missing size", sizeOpt.isPresent());
       if (shardName.endsWith("1")) {
         assertEquals("size", 10, ((Number) sizeOpt.get()).intValue());
@@ -126,7 +128,7 @@ public class BuildersTest extends SolrTestCaseJ4 {
                 assertTrue("replica metrics", metricsOpt.isPresent());
                 ReplicaMetrics metrics = metricsOpt.get();
                 Optional<Double> replicaSizeOpt =
-                    metrics.getReplicaMetric(ReplicaMetricImpl.INDEX_SIZE_GB);
+                    metrics.getReplicaMetric(BuiltInMetrics.REPLICA_INDEX_SIZE_GB);
                 assertTrue("missing size", replicaSizeOpt.isPresent());
                 if (shardName.endsWith("1")) {
                   assertEquals("size", 10, ((Number) replicaSizeOpt.get()).intValue());