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

[lucene-solr] branch jira/solr-15019 updated: SOLR-15019: Remove unnecessary API made obsolete by the use of NodeMetric-s.

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

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


The following commit(s) were added to refs/heads/jira/solr-15019 by this push:
     new f08da04a SOLR-15019: Remove unnecessary API made obsolete by the use of NodeMetric-s.
f08da04a is described below

commit f08da04ae831fcdecb2737e6249044c4a40940e5
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue Dec 22 19:22:14 2020 +0100

    SOLR-15019: Remove unnecessary API made obsolete by the use of NodeMetric-s.
---
 .../org/apache/solr/cluster/SolrCollection.java    |  10 +-
 .../solr/cluster/placement/AttributeFetcher.java   |  65 +-----------
 .../solr/cluster/placement/AttributeValues.java    |  43 --------
 .../solr/cluster/placement/MetricAttribute.java    | 114 +++------------------
 .../apache/solr/cluster/placement/NodeMetric.java  | 110 +++++---------------
 .../solr/cluster/placement/ReplicaMetric.java      |  17 +--
 .../placement/impl/AttributeFetcherImpl.java       |  13 +--
 .../placement/impl/AttributeValuesImpl.java        |  12 +--
 .../placement/impl/CollectionMetricsBuilder.java   |   2 +-
 .../MetricAttributeImpl.java}                      |  39 +++----
 .../{NodeMetric.java => impl/NodeMetricImpl.java}  |  43 ++++----
 .../ReplicaMetricImpl.java}                        |  16 +--
 .../plugins/AffinityPlacementFactory.java          |  19 ++--
 .../plugins/MinimizeCoresPlacementFactory.java     |   7 +-
 .../cluster/placement/AttributeFetcherForTest.java |  32 +-----
 .../apache/solr/cluster/placement/Builders.java    |  12 ++-
 .../solr/cluster/placement/BuildersTest.java       |  20 ++--
 .../impl/PlacementPluginIntegrationTest.java       |  61 +++++------
 18 files changed, 163 insertions(+), 472 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 d22560a..ea7ea45 100644
--- a/solr/core/src/java/org/apache/solr/cluster/SolrCollection.java
+++ b/solr/core/src/java/org/apache/solr/cluster/SolrCollection.java
@@ -18,7 +18,6 @@
 package org.apache.solr.cluster;
 
 import org.apache.solr.cluster.placement.AttributeFetcher;
-import org.apache.solr.cluster.placement.AttributeValues;
 import org.apache.solr.cluster.placement.PlacementPlugin;
 import org.apache.solr.cluster.placement.PlacementRequest;
 
@@ -69,12 +68,11 @@ public interface SolrCollection {
      * <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
+     * <p>For example if a collection is to be placed only on nodes using located in a specific availability zone, it can be
+     * identified as such using some custom property (collection property could for example be called "availabilityZone" and have
+     * value "az1" 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}.
+     * of this collection on {@link Node}'s for which this attribute is non empty and equal.
      */
   String getCustomProperty(String customPropertyName);
 
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 aaa6377..14051d7 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
@@ -27,41 +27,6 @@ import java.util.Set;
  */
 public interface AttributeFetcher {
   /**
-   * Request the number of cores on each node. To get the value use {@link AttributeValues#getCoresCount(Node)}
-   */
-  default AttributeFetcher requestNodeCoresCount() {
-    return requestNodeMetric(NodeMetric.NUM_CORES);
-  }
-
-  /**
-   * Request the free disk size on each node. To get the value use {@link AttributeValues#getFreeDisk(Node)}
-   */
-  default AttributeFetcher requestNodeFreeDisk() {
-    return requestNodeMetric(NodeMetric.FREE_DISK_GB);
-  }
-
-  /**
-   * Request the total disk size on each node. To get the value use {@link AttributeValues#getTotalDisk(Node)}
-   */
-  default AttributeFetcher requestNodeTotalDisk() {
-    return requestNodeMetric(NodeMetric.TOTAL_DISK_GB);
-  }
-
-  /**
-   * Request the heap usage on each node. To get the value use {@link AttributeValues#getHeapUsage(Node)}
-   */
-  default AttributeFetcher requestNodeHeapUsage() {
-    return requestNodeMetric(NodeMetric.HEAP_USAGE);
-  }
-
-  /**
-   * Request the system load average on each node. To get the value use {@link AttributeValues#getSystemLoadAverage(Node)}
-   */
-  default AttributeFetcher requestNodeSystemLoadAverage() {
-    return requestNodeMetric(NodeMetric.SYSLOAD_AVG);
-  }
-
-  /**
    * Request a given system property on each node. To get the value use {@link AttributeValues#getSystemProperty(Node, String)}
    * @param name system property name
    */
@@ -80,16 +45,6 @@ public interface AttributeFetcher {
   AttributeFetcher requestNodeMetric(NodeMetric<?> metric);
 
   /**
-   * Rfrom any metric registry on each node, using a fully-qualified metric key,
-   * for example <code>solr.jvm:system.properties:user.name</code>.
-   * To get the value use {@link AttributeValues#getNodeMetric(Node, String)}
-   * @param metricKey fully-qualified metric key
-   */
-  default AttributeFetcher requestNodeMetric(String metricKey) {
-    return requestNodeMetric(new NodeMetric<>(metricKey));
-  }
-
-  /**
    * Request collection-level metrics. To get the values use {@link AttributeValues#getCollectionMetrics(String)}.
    * Note that this request will fetch information from nodes relevant to the collection
    * replicas and not the ones specified in {@link #fetchFrom(Set)} (though they may overlap).
@@ -107,27 +62,9 @@ public interface AttributeFetcher {
 
   /**
    * 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 #requestNodeMetric(String)}.
+   * (those requested for example using {@link #requestNodeMetric(NodeMetric)}.
    *
    * @return An instance allowing retrieval of all attributed that could be fetched.
    */
   AttributeValues fetchAttributes();
-
-  /**
-   * Registry options for {@link Node} metrics.
-   */
-  enum NodeMetricRegistry {
-    /**
-     * corresponds to solr.node
-     */
-    SOLR_NODE,
-    /**
-     * corresponds to solr.jvm
-     */
-    SOLR_JVM,
-    /**
-     * corresponds to solr.jetty
-     */
-    SOLR_JETTY
-  }
 }
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 9fe0a1b..43e51e5 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
@@ -23,41 +23,6 @@ import java.util.Optional;
 
 public interface AttributeValues {
   /**
-   * For the given node: number of cores
-   */
-  default Optional<Integer> getCoresCount(Node node) {
-    return getNodeMetric(node, NodeMetric.NUM_CORES);
-  }
-
-  /**
-   * For the given node: Free disk size in Gigabytes of the partition on which cores are stored
-   */
-  default Optional<Double> getFreeDisk(Node node) {
-    return getNodeMetric(node, NodeMetric.FREE_DISK_GB);
-  }
-
-  /**
-   * For the given node: Total disk size in Gigabytes of the partition on which cores are stored
-   */
-  default Optional<Double> getTotalDisk(Node node) {
-    return getNodeMetric(node, NodeMetric.TOTAL_DISK_GB);
-  }
-
-  /**
-   * For the given node: Percentage between 0 and 100 of used heap over max heap
-   */
-  default Optional<Double> getHeapUsage(Node node) {
-    return getNodeMetric(node, NodeMetric.HEAP_USAGE);
-  }
-
-  /**
-   * For the given node: matches {@link java.lang.management.OperatingSystemMXBean#getSystemLoadAverage()}
-   */
-  default Optional<Double> getSystemLoadAverage(Node node) {
-    return getNodeMetric(node, NodeMetric.HEAP_USAGE);
-  }
-
-  /**
    * For the given node: system property value (system properties are passed to Java using {@code -Dname=value}
    */
   Optional<String> getSystemProperty(Node node, String name);
@@ -72,14 +37,6 @@ public interface AttributeValues {
    */
   <T> Optional<T> getNodeMetric(Node node, NodeMetric<T> metric);
 
-
-  /**
-   * Get any metric using a fully-qualified metric key.
-   */
-  default Optional<Object> getNodeMetric(Node node, String metricKey) {
-    return getNodeMetric(node, new NodeMetric<>(metricKey));
-  }
-
   /**
    * Get collection metrics.
    */
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/MetricAttribute.java b/solr/core/src/java/org/apache/solr/cluster/placement/MetricAttribute.java
index a638660..c587b9e 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/MetricAttribute.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/MetricAttribute.java
@@ -16,7 +16,6 @@
  */
 package org.apache.solr.cluster.placement;
 
-import java.util.Objects;
 import java.util.function.Function;
 
 /**
@@ -24,120 +23,31 @@ import java.util.function.Function;
  * internal metric name and the desired format/unit conversion. Generic type
  * defines the type of converted values of this attribute.
  */
-public class MetricAttribute<T> {
-
-  public static final double GB = 1024 * 1024 * 1024;
-
-  /**
-   * Identity converter. It returns the raw value unchanged IFF
-   * the value's type can be cast to the generic type of this attribute,
-   * otherwise it returns null.
-   */
-  @SuppressWarnings("unchecked")
-  public final Function<Object, T> IDENTITY_CONVERTER = v -> {
-    try {
-      return (T) v;
-    } catch (ClassCastException cce) {
-      return null;
-    }
-  };
+public interface MetricAttribute<T> {
 
   /**
-   * Bytes to gigabytes converter. Supports converting number or string
-   * representations of raw values expressed in bytes.
-   */
-  @SuppressWarnings("unchecked")
-  public static final Function<Object, Double> BYTES_TO_GB_CONVERTER = v -> {
-    double sizeInBytes;
-    if (!(v instanceof Number)) {
-      if (v == null) {
-        return null;
-      }
-      try {
-        sizeInBytes = Double.valueOf(String.valueOf(v)).doubleValue();
-      } catch (Exception nfe) {
-        return null;
-      }
-    } else {
-      sizeInBytes = ((Number) v).doubleValue();
-    }
-    return sizeInBytes / GB;
-  };
-
-  protected final String name;
-  protected final String internalName;
-  protected final Function<Object, T> converter;
-
-  /**
-   * Create a metric attribute.
-   * @param name short-hand name that identifies this attribute.
-   * @param internalName internal name of a Solr metric.
+   * Return the short-hand name that identifies this attribute.
    */
-  public MetricAttribute(String name, String internalName) {
-    this(name, internalName, null);
-  }
+  String getName();
 
   /**
-   * Create a metric attribute.
-   * @param name short-hand name that identifies this attribute.
-   * @param internalName internal name of a Solr metric.
-   * @param converter optional raw value converter. If null then
-   *                  {@link #IDENTITY_CONVERTER} will be used.
-   */
-  public MetricAttribute(String name, String internalName, Function<Object, T> converter) {
-    Objects.requireNonNull(name);
-    Objects.requireNonNull(internalName);
-    this.name = name;
-    this.internalName = internalName;
-    if (converter == null) {
-      this.converter = IDENTITY_CONVERTER;
-    } else {
-      this.converter = converter;
-    }
-  }
-
-  /**
-   * Return the short-hand name that identifies this attribute.
+   * Return the internal name of a Solr metric associated with this attribute.
    */
-  public String getName() {
-    return name;
-  }
+  String getInternalName();
 
   /**
-   * Return the internal name of a Solr metric associated with this attribute.
+   * Conversion function to convert formats/units of raw values.
    */
-  public String getInternalName() {
-    return internalName;
-  }
+  Function<Object, T> getConverter();
 
   /**
-   * Convert raw value. This may involve changing the type or units.
+   * Convert raw value. This may involve changing value type or units.
+   * Default implementation simply applies the converter function
+   * returned by {@link #getConverter()}.
    * @param value raw value
    * @return converted value
    */
-  public T convert(Object value) {
-    return converter.apply(value);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    MetricAttribute<?> that = (MetricAttribute<?>) o;
-    return name.equals(that.name) && internalName.equals(that.internalName) && converter.equals(that.converter);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(name, internalName, converter);
-  }
-
-  @Override
-  public String toString() {
-    return name + "(" + internalName + ")";
+  default T convert(Object value) {
+    return getConverter().apply(value);
   }
 }
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 d4f382e..fba5aed 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
@@ -17,93 +17,35 @@
 
 package org.apache.solr.cluster.placement;
 
-import org.apache.solr.common.cloud.rule.ImplicitSnitch;
-
-import java.util.Objects;
-import java.util.function.Function;
-
 /**
  * Node metric identifier, corresponding
  * to a node-level metric registry and the internal metric name.
+ * Alternatively this identified may use a fully-qualified metric key,
+ * in which case the registry is set to null.
  */
-public class NodeMetric<T> extends MetricAttribute<T> {
-
-  /** Total disk space in GB. */
-  public static final NodeMetric<Double> TOTAL_DISK_GB = new NodeMetric<>("totalDisk",
-      AttributeFetcher.NodeMetricRegistry.SOLR_NODE, "CONTAINER.fs.totalSpace", BYTES_TO_GB_CONVERTER);
-
-  /** Free (usable) disk space in GB. */
-  public static final NodeMetric<Double> FREE_DISK_GB = new NodeMetric<>("freeDisk",
-      AttributeFetcher.NodeMetricRegistry.SOLR_NODE, "CONTAINER.fs.usableSpace", BYTES_TO_GB_CONVERTER);
-
-  /** Number of all cores. */
-  public static final NodeMetric<Integer> NUM_CORES = new NodeMetric<>(ImplicitSnitch.CORES);
-  public static final NodeMetric<Double> HEAP_USAGE = new NodeMetric<>(ImplicitSnitch.HEAPUSAGE);
-
-  /** System load average. */
-  public static final NodeMetric<Double> SYSLOAD_AVG =
-      new NodeMetric<>("sysLoadAvg", AttributeFetcher.NodeMetricRegistry.SOLR_JVM, "os.systemLoadAverage");
-
-  /** Number of available processors. */
-  public static final NodeMetric<Integer> AVAILABLE_PROCESSORS =
-      new NodeMetric<>("availableProcessors", AttributeFetcher.NodeMetricRegistry.SOLR_JVM, "os.availableProcessors");
-
-  private final AttributeFetcher.NodeMetricRegistry registry;
-
-  public NodeMetric(String name, AttributeFetcher.NodeMetricRegistry registry, String internalName) {
-    this(name, registry, internalName, null);
-  }
-
-  public NodeMetric(String name, AttributeFetcher.NodeMetricRegistry registry, String internalName, Function<Object, T> converter) {
-    super(name, internalName, converter);
-    Objects.requireNonNull(registry);
-    this.registry = registry;
-  }
-
-  public NodeMetric(String key) {
-    this(key, null);
-  }
-
-  public NodeMetric(String key, Function<Object, T> converter) {
-    super(key, key, converter);
-    this.registry = null;
-  }
-
-  public AttributeFetcher.NodeMetricRegistry getRegistry() {
-    return registry;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    if (!super.equals(o)) {
-      return false;
-    }
-    NodeMetric<?> that = (NodeMetric<?>) o;
-    return registry == that.registry;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(super.hashCode(), registry);
-  }
-
-  @Override
-  public String toString() {
-    if (registry != null) {
-      return "NodeMetric{" +
-          "name='" + name + '\'' +
-          ", internalName='" + internalName + '\'' +
-          ", converter=" + converter +
-          ", registry=" + registry +
-          '}';
-    } else {
-      return "NodeMetric{key=" + internalName + "}";
-    }
+public interface NodeMetric<T> extends MetricAttribute<T> {
+
+  /**
+   * Metric registry. If this metric identifier uses a fully-qualified
+   * metric key instead, then this method will return null.
+   */
+  Registry getRegistry();
+
+  /**
+   * Registry options for node metrics.
+   */
+  enum Registry {
+    /**
+     * corresponds to solr.node
+     */
+    SOLR_NODE,
+    /**
+     * corresponds to solr.jvm
+     */
+    SOLR_JVM,
+    /**
+     * corresponds to solr.jetty
+     */
+    SOLR_JETTY
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/ReplicaMetric.java b/solr/core/src/java/org/apache/solr/cluster/placement/ReplicaMetric.java
index 8d9c25d..368c313 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/ReplicaMetric.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/ReplicaMetric.java
@@ -16,24 +16,9 @@
  */
 package org.apache.solr.cluster.placement;
 
-import java.util.function.Function;
-
 /**
  * Replica metric identifier, corresponding to one of the
  * internal replica-level metric names (as reported in <code>solr.core.[collection].[replica]</code> registry)
  */
-public class ReplicaMetric<T> extends MetricAttribute<T> {
-
-  public static final ReplicaMetric<Double> INDEX_SIZE_GB = new ReplicaMetric<>("sizeGB", "INDEX.sizeInBytes", BYTES_TO_GB_CONVERTER);
-
-  public static final ReplicaMetric<Double> QUERY_RATE_1MIN = new ReplicaMetric<>("queryRate", "QUERY./select.requestTimes:1minRate");
-  public static final ReplicaMetric<Double> UPDATE_RATE_1MIN = new ReplicaMetric<>("updateRate", "UPDATE./update.requestTimes:1minRate");
-
-  public ReplicaMetric(String name, String internalName) {
-    super(name, internalName);
-  }
-
-  public ReplicaMetric(String name, String internalName, Function<Object, T> converter) {
-    super(name, internalName, converter);
-  }
+public interface ReplicaMetric<T> extends MetricAttribute<T> {
 }
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 a5837a5..b4bb971 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
@@ -39,14 +39,13 @@ import java.util.*;
 import java.util.function.BiConsumer;
 import java.util.stream.Collectors;
 
+/**
+ * Implementation of {@link AttributeFetcher} that uses {@link SolrCloudManager}
+ * to access Solr cluster details.
+ */
 public class AttributeFetcherImpl implements AttributeFetcher {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  boolean requestedNodeCoreCount;
-  boolean requestedNodeFreeDisk;
-  boolean requestedNodeTotalDisk;
-  boolean requestedNodeHeapUsage;
-  boolean requestedNodeSystemLoadAverage;
   Set<String> requestedNodeSystemSnitchTags = new HashSet<>();
   Set<NodeMetric<?>> requestedNodeMetricSnitchTags = new HashSet<>();
   Map<SolrCollection, Set<ReplicaMetric<?>>> requestedCollectionMetrics = new HashMap<>();
@@ -189,12 +188,14 @@ public class AttributeFetcherImpl implements AttributeFetcher {
         metricSnitchToNodeToValue, collectionMetrics);
   }
 
-  private static SolrInfoBean.Group getGroupFromMetricRegistry(NodeMetricRegistry registry) {
+  private static SolrInfoBean.Group getGroupFromMetricRegistry(NodeMetric.Registry registry) {
     switch (registry) {
       case SOLR_JVM:
         return SolrInfoBean.Group.jvm;
       case SOLR_NODE:
         return SolrInfoBean.Group.node;
+      case SOLR_JETTY:
+        return SolrInfoBean.Group.jetty;
       default:
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unsupported registry value " + registry);
     }
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 6101e3d..873bd3d 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,6 +25,9 @@ import org.apache.solr.cluster.placement.NodeMetric;
 import java.util.Map;
 import java.util.Optional;
 
+/**
+ * Implementation of {@link AttributeValues} used by {@link AttributeFetcherImpl}.
+ */
 public class AttributeValuesImpl implements AttributeValues {
   // sysprop (or sysenv) name / node -> value
   final Map<String, Map<Node, String>> systemSnitchToNodeToValue;
@@ -70,15 +73,6 @@ public class AttributeValuesImpl implements AttributeValues {
   }
 
   @Override
-  public Optional<Object> getNodeMetric(Node node, String metricKey) {
-    Map<Node, Object> nodeToValue = metricSnitchToNodeToValue.get(new NodeMetric<>(metricKey));
-    if (nodeToValue == null) {
-      return Optional.empty();
-    }
-    return Optional.ofNullable(nodeToValue.get(node));
-  }
-
-  @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/CollectionMetricsBuilder.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/CollectionMetricsBuilder.java
index 00bfdf2..0f88feb 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/CollectionMetricsBuilder.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/CollectionMetricsBuilder.java
@@ -26,7 +26,7 @@ import java.util.Map;
 import java.util.Optional;
 
 /**
- *
+ * Builder class for constructing instances of {@link CollectionMetrics}.
  */
 public class CollectionMetricsBuilder {
 
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/MetricAttribute.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/MetricAttributeImpl.java
similarity index 76%
copy from solr/core/src/java/org/apache/solr/cluster/placement/MetricAttribute.java
copy to solr/core/src/java/org/apache/solr/cluster/placement/impl/MetricAttributeImpl.java
index a638660..60e55f1 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/MetricAttribute.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/MetricAttributeImpl.java
@@ -14,17 +14,17 @@
  * 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.impl;
+
+import org.apache.solr.cluster.placement.MetricAttribute;
 
 import java.util.Objects;
 import java.util.function.Function;
 
 /**
- * Metric-related attribute of a node or replica. It defines a short symbolic name of the metric, the corresponding
- * internal metric name and the desired format/unit conversion. Generic type
- * defines the type of converted values of this attribute.
+ * Base class for {@link MetricAttribute} implementations.
  */
-public class MetricAttribute<T> {
+public class MetricAttributeImpl<T> implements MetricAttribute<T> {
 
   public static final double GB = 1024 * 1024 * 1024;
 
@@ -73,7 +73,7 @@ public class MetricAttribute<T> {
    * @param name short-hand name that identifies this attribute.
    * @param internalName internal name of a Solr metric.
    */
-  public MetricAttribute(String name, String internalName) {
+  public MetricAttributeImpl(String name, String internalName) {
     this(name, internalName, null);
   }
 
@@ -84,7 +84,7 @@ public class MetricAttribute<T> {
    * @param converter optional raw value converter. If null then
    *                  {@link #IDENTITY_CONVERTER} will be used.
    */
-  public MetricAttribute(String name, String internalName, Function<Object, T> converter) {
+  public MetricAttributeImpl(String name, String internalName, Function<Object, T> converter) {
     Objects.requireNonNull(name);
     Objects.requireNonNull(internalName);
     this.name = name;
@@ -96,27 +96,19 @@ public class MetricAttribute<T> {
     }
   }
 
-  /**
-   * Return the short-hand name that identifies this attribute.
-   */
+  @Override
   public String getName() {
     return name;
   }
 
-  /**
-   * Return the internal name of a Solr metric associated with this attribute.
-   */
+  @Override
   public String getInternalName() {
     return internalName;
   }
 
-  /**
-   * Convert raw value. This may involve changing the type or units.
-   * @param value raw value
-   * @return converted value
-   */
-  public T convert(Object value) {
-    return converter.apply(value);
+  @Override
+  public Function<Object, T> getConverter() {
+    return converter;
   }
 
   @Override
@@ -128,7 +120,7 @@ public class MetricAttribute<T> {
       return false;
     }
     MetricAttribute<?> that = (MetricAttribute<?>) o;
-    return name.equals(that.name) && internalName.equals(that.internalName) && converter.equals(that.converter);
+    return name.equals(that.getName()) && internalName.equals(that.getInternalName()) && converter.equals(that.getConverter());
   }
 
   @Override
@@ -138,6 +130,9 @@ public class MetricAttribute<T> {
 
   @Override
   public String toString() {
-    return name + "(" + internalName + ")";
+    return getClass().getSimpleName() + "{" +
+        "name=" + name +
+        ", internalName=" + internalName +
+        "}";
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/NodeMetric.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/NodeMetricImpl.java
similarity index 57%
copy from solr/core/src/java/org/apache/solr/cluster/placement/NodeMetric.java
copy to solr/core/src/java/org/apache/solr/cluster/placement/impl/NodeMetricImpl.java
index d4f382e..f98f441 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/NodeMetric.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/NodeMetricImpl.java
@@ -15,8 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.solr.cluster.placement;
+package org.apache.solr.cluster.placement.impl;
 
+import org.apache.solr.cluster.placement.NodeMetric;
 import org.apache.solr.common.cloud.rule.ImplicitSnitch;
 
 import java.util.Objects;
@@ -26,50 +27,50 @@ import java.util.function.Function;
  * Node metric identifier, corresponding
  * to a node-level metric registry and the internal metric name.
  */
-public class NodeMetric<T> extends MetricAttribute<T> {
+public class NodeMetricImpl<T> extends MetricAttributeImpl<T> implements NodeMetric<T> {
 
   /** Total disk space in GB. */
-  public static final NodeMetric<Double> TOTAL_DISK_GB = new NodeMetric<>("totalDisk",
-      AttributeFetcher.NodeMetricRegistry.SOLR_NODE, "CONTAINER.fs.totalSpace", BYTES_TO_GB_CONVERTER);
+  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 NodeMetric<Double> FREE_DISK_GB = new NodeMetric<>("freeDisk",
-      AttributeFetcher.NodeMetricRegistry.SOLR_NODE, "CONTAINER.fs.usableSpace", BYTES_TO_GB_CONVERTER);
+  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 NodeMetric<Integer> NUM_CORES = new NodeMetric<>(ImplicitSnitch.CORES);
-  public static final NodeMetric<Double> HEAP_USAGE = new NodeMetric<>(ImplicitSnitch.HEAPUSAGE);
+  public static final NodeMetricImpl<Integer> NUM_CORES = new NodeMetricImpl<>(ImplicitSnitch.CORES);
+  public static final NodeMetricImpl<Double> HEAP_USAGE = new NodeMetricImpl<>(ImplicitSnitch.HEAPUSAGE);
 
   /** System load average. */
-  public static final NodeMetric<Double> SYSLOAD_AVG =
-      new NodeMetric<>("sysLoadAvg", AttributeFetcher.NodeMetricRegistry.SOLR_JVM, "os.systemLoadAverage");
+  public static final NodeMetricImpl<Double> SYSLOAD_AVG =
+      new NodeMetricImpl<>("sysLoadAvg", Registry.SOLR_JVM, "os.systemLoadAverage");
 
   /** Number of available processors. */
-  public static final NodeMetric<Integer> AVAILABLE_PROCESSORS =
-      new NodeMetric<>("availableProcessors", AttributeFetcher.NodeMetricRegistry.SOLR_JVM, "os.availableProcessors");
+  public static final NodeMetricImpl<Integer> AVAILABLE_PROCESSORS =
+      new NodeMetricImpl<>("availableProcessors", Registry.SOLR_JVM, "os.availableProcessors");
 
-  private final AttributeFetcher.NodeMetricRegistry registry;
+  private final Registry registry;
 
-  public NodeMetric(String name, AttributeFetcher.NodeMetricRegistry registry, String internalName) {
+  public NodeMetricImpl(String name, Registry registry, String internalName) {
     this(name, registry, internalName, null);
   }
 
-  public NodeMetric(String name, AttributeFetcher.NodeMetricRegistry registry, String internalName, Function<Object, T> converter) {
+  public NodeMetricImpl(String name, Registry registry, String internalName, Function<Object, T> converter) {
     super(name, internalName, converter);
     Objects.requireNonNull(registry);
     this.registry = registry;
   }
 
-  public NodeMetric(String key) {
+  public NodeMetricImpl(String key) {
     this(key, null);
   }
 
-  public NodeMetric(String key, Function<Object, T> converter) {
+  public NodeMetricImpl(String key, Function<Object, T> converter) {
     super(key, key, converter);
     this.registry = null;
   }
 
-  public AttributeFetcher.NodeMetricRegistry getRegistry() {
+  public Registry getRegistry() {
     return registry;
   }
 
@@ -84,7 +85,7 @@ public class NodeMetric<T> extends MetricAttribute<T> {
     if (!super.equals(o)) {
       return false;
     }
-    NodeMetric<?> that = (NodeMetric<?>) o;
+    NodeMetricImpl<?> that = (NodeMetricImpl<?>) o;
     return registry == that.registry;
   }
 
@@ -96,14 +97,14 @@ public class NodeMetric<T> extends MetricAttribute<T> {
   @Override
   public String toString() {
     if (registry != null) {
-      return "NodeMetric{" +
+      return "NodeMetricImpl{" +
           "name='" + name + '\'' +
           ", internalName='" + internalName + '\'' +
           ", converter=" + converter +
           ", registry=" + registry +
           '}';
     } else {
-      return "NodeMetric{key=" + internalName + "}";
+      return "NodeMetricImpl{key=" + internalName + "}";
     }
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/ReplicaMetric.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/ReplicaMetricImpl.java
similarity index 58%
copy from solr/core/src/java/org/apache/solr/cluster/placement/ReplicaMetric.java
copy to solr/core/src/java/org/apache/solr/cluster/placement/impl/ReplicaMetricImpl.java
index 8d9c25d..27afa72 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/ReplicaMetric.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/ReplicaMetricImpl.java
@@ -14,7 +14,9 @@
  * 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.impl;
+
+import org.apache.solr.cluster.placement.ReplicaMetric;
 
 import java.util.function.Function;
 
@@ -22,18 +24,18 @@ import java.util.function.Function;
  * Replica metric identifier, corresponding to one of the
  * internal replica-level metric names (as reported in <code>solr.core.[collection].[replica]</code> registry)
  */
-public class ReplicaMetric<T> extends MetricAttribute<T> {
+public class ReplicaMetricImpl<T> extends MetricAttributeImpl<T> implements ReplicaMetric<T> {
 
-  public static final ReplicaMetric<Double> INDEX_SIZE_GB = new ReplicaMetric<>("sizeGB", "INDEX.sizeInBytes", BYTES_TO_GB_CONVERTER);
+  public static final ReplicaMetricImpl<Double> INDEX_SIZE_GB = new ReplicaMetricImpl<>("sizeGB", "INDEX.sizeInBytes", BYTES_TO_GB_CONVERTER);
 
-  public static final ReplicaMetric<Double> QUERY_RATE_1MIN = new ReplicaMetric<>("queryRate", "QUERY./select.requestTimes:1minRate");
-  public static final ReplicaMetric<Double> UPDATE_RATE_1MIN = new ReplicaMetric<>("updateRate", "UPDATE./update.requestTimes:1minRate");
+  public static final ReplicaMetricImpl<Double> QUERY_RATE_1MIN = new ReplicaMetricImpl<>("queryRate", "QUERY./select.requestTimes:1minRate");
+  public static final ReplicaMetricImpl<Double> UPDATE_RATE_1MIN = new ReplicaMetricImpl<>("updateRate", "UPDATE./update.requestTimes:1minRate");
 
-  public ReplicaMetric(String name, String internalName) {
+  public ReplicaMetricImpl(String name, String internalName) {
     super(name, internalName);
   }
 
-  public ReplicaMetric(String name, String internalName, Function<Object, T> converter) {
+  public ReplicaMetricImpl(String name, String internalName, Function<Object, T> converter) {
     super(name, internalName, converter);
   }
 }
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 b40d300..9c50289 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
@@ -21,6 +21,7 @@ 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.cluster.placement.impl.NodeMetricImpl;
 import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.slf4j.Logger;
@@ -194,7 +195,9 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
 
       // Request all needed attributes
       attributeFetcher.requestNodeSystemProperty(AVAILABILITY_ZONE_SYSPROP).requestNodeSystemProperty(REPLICA_TYPE_SYSPROP);
-      attributeFetcher.requestNodeCoresCount().requestNodeFreeDisk();
+      attributeFetcher
+          .requestNodeMetric(NodeMetricImpl.NUM_CORES)
+          .requestNodeMetric(NodeMetricImpl.FREE_DISK_GB);
       attributeFetcher.fetchFrom(nodes);
       final AttributeValues attrValues = attributeFetcher.fetchAttributes();
 
@@ -306,21 +309,21 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
 
       for (Node node : nodes) {
         // Exclude nodes with unknown or too small disk free space
-        if (attrValues.getFreeDisk(node).isEmpty()) {
+        if (attrValues.getNodeMetric(node, NodeMetricImpl.FREE_DISK_GB).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 (attrValues.getNodeMetric(node, NodeMetricImpl.FREE_DISK_GB).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);
+            log.warn("Node {} free disk ({}GB) lower than configured minimum {}GB, excluding it from placement decisions.", node.getName(), attrValues.getNodeMetric(node, NodeMetricImpl.FREE_DISK_GB).get(), minimalFreeDiskGB);
           }
           continue;
         }
 
-        if (attrValues.getCoresCount(node).isEmpty()) {
+        if (attrValues.getNodeMetric(node, NodeMetricImpl.NUM_CORES).isEmpty()) {
           if (log.isWarnEnabled()) {
             log.warn("Unknown number of cores on node {}, excluding it from placement decisions.", node.getName());
           }
@@ -328,7 +331,7 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
           continue;
         }
 
-        Integer coresCount = attrValues.getCoresCount(node).get();
+        Integer coresCount = attrValues.getNodeMetric(node, NodeMetricImpl.NUM_CORES).get();
         coresOnNodes.put(node, coresCount);
 
         String supportedReplicaTypes = attrValues.getSystemProperty(node, REPLICA_TYPE_SYSPROP).isPresent() ? attrValues.getSystemProperty(node, REPLICA_TYPE_SYSPROP).get() : null;
@@ -552,8 +555,8 @@ public class AffinityPlacementFactory implements PlacementPluginFactory<Affinity
       @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() < prioritizedFreeDiskGB;
-        boolean bHasLowFreeSpace = attrValues.getFreeDisk(b).get() < prioritizedFreeDiskGB;
+        boolean aHasLowFreeSpace = attrValues.getNodeMetric(a, NodeMetricImpl.FREE_DISK_GB).get() < prioritizedFreeDiskGB;
+        boolean bHasLowFreeSpace = attrValues.getNodeMetric(b, NodeMetricImpl.FREE_DISK_GB).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/java/org/apache/solr/cluster/placement/plugins/MinimizeCoresPlacementFactory.java b/solr/core/src/java/org/apache/solr/cluster/placement/plugins/MinimizeCoresPlacementFactory.java
index 5b3d2aa..bb1e762 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
@@ -31,6 +31,7 @@ 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.cluster.placement.impl.NodeMetricImpl;
 import org.apache.solr.common.util.SuppressForbidden;
 
 /**
@@ -66,17 +67,17 @@ public class MinimizeCoresPlacementFactory implements PlacementPluginFactory<Pla
 
       Set<Node> nodes = request.getTargetNodes();
 
-      attributeFetcher.requestNodeCoresCount();
+      attributeFetcher.requestNodeMetric(NodeMetricImpl.NUM_CORES);
       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()) {
+        if (attrValues.getNodeMetric(node, NodeMetricImpl.NUM_CORES).isEmpty()) {
           throw new PlacementException("Can't get number of cores in " + node);
         }
-        nodesByCores.put(attrValues.getCoresCount(node).get(), node);
+        nodesByCores.put(attrValues.getNodeMetric(node, NodeMetricImpl.NUM_CORES).get(), node);
       }
 
       Set<ReplicaPlacement> replicaPlacements = new HashSet<>(totalReplicasPerShard * request.getShardNames().size());
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/AttributeFetcherForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/AttributeFetcherForTest.java
index e7903b9..bcc300b 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/AttributeFetcherForTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/AttributeFetcherForTest.java
@@ -31,31 +31,6 @@ public class AttributeFetcherForTest implements AttributeFetcher {
   }
 
   @Override
-  public AttributeFetcher requestNodeCoresCount() {
-    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;
   }
@@ -67,7 +42,7 @@ public class AttributeFetcherForTest implements AttributeFetcher {
 
   @Override
   public AttributeFetcher requestNodeMetric(NodeMetric<?> metric) {
-    throw new UnsupportedOperationException("Not yet implemented...");
+    return this;
   }
 
   @Override
@@ -81,11 +56,6 @@ public class AttributeFetcherForTest implements AttributeFetcher {
   }
 
   @Override
-  public AttributeFetcher requestNodeMetric(String metricKey) {
-    throw new UnsupportedOperationException("Not yet implemented...");
-  }
-
-  @Override
   public AttributeValues fetchAttributes() {
     return attributeValues;
   }
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/Builders.java b/solr/core/src/test/org/apache/solr/cluster/placement/Builders.java
index 2afad19..44807e5 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/Builders.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/Builders.java
@@ -21,6 +21,8 @@ import org.apache.solr.cluster.*;
 import org.apache.solr.cluster.placement.impl.AttributeFetcherImpl;
 import org.apache.solr.cluster.placement.impl.AttributeValuesImpl;
 import org.apache.solr.cluster.placement.impl.CollectionMetricsBuilder;
+import org.apache.solr.cluster.placement.impl.NodeMetricImpl;
+import org.apache.solr.cluster.placement.impl.ReplicaMetricImpl;
 import org.apache.solr.common.util.Pair;
 import org.junit.Assert;
 
@@ -103,15 +105,15 @@ public class Builders {
         Node node = nodeBuilder.build();
 
         if (nodeBuilder.getCoreCount() != null) {
-          metrics.computeIfAbsent(NodeMetric.NUM_CORES, n -> new HashMap<>())
+          metrics.computeIfAbsent(NodeMetricImpl.NUM_CORES, n -> new HashMap<>())
               .put(node, nodeBuilder.getCoreCount());
         }
         if (nodeBuilder.getFreeDiskGB() != null) {
-          metrics.computeIfAbsent(NodeMetric.FREE_DISK_GB, n -> new HashMap<>())
+          metrics.computeIfAbsent(NodeMetricImpl.FREE_DISK_GB, n -> new HashMap<>())
               .put(node, nodeBuilder.getFreeDiskGB());
         }
         if (nodeBuilder.getTotalDiskGB() != null) {
-          metrics.computeIfAbsent(NodeMetric.TOTAL_DISK_GB, n -> new HashMap<>())
+          metrics.computeIfAbsent(NodeMetricImpl.TOTAL_DISK_GB, n -> new HashMap<>())
               .put(node, nodeBuilder.getTotalDiskGB());
         }
         if (nodeBuilder.getSysprops() != null) {
@@ -129,7 +131,7 @@ public class Builders {
       }
 
       if (!collectionBuilders.isEmpty()) {
-        Map<Node, Object> nodeToCoreCount = metrics.computeIfAbsent(NodeMetric.NUM_CORES, n -> new HashMap<>());
+        Map<Node, Object> nodeToCoreCount = metrics.computeIfAbsent(NodeMetricImpl.NUM_CORES, n -> new HashMap<>());
         collectionBuilders.forEach(builder -> {
           collectionMetrics.put(builder.collectionName, builder.collectionMetricsBuilder.build());
           SolrCollection collection = builder.build();
@@ -316,7 +318,7 @@ public class Builders {
             CollectionMetricsBuilder.ReplicaMetricsBuilder replicaMetricsBuilder = new CollectionMetricsBuilder.ReplicaMetricsBuilder();
             shardMetricsBuilder.getReplicaMetricsBuilders().put(replicaName, replicaMetricsBuilder);
             if (initialSizeGBPerShard != null) {
-              replicaMetricsBuilder.addMetric(ReplicaMetric.INDEX_SIZE_GB, initialSizeGBPerShard.get(shardNumber - 1) * ReplicaMetric.GB);
+              replicaMetricsBuilder.addMetric(ReplicaMetricImpl.INDEX_SIZE_GB, initialSizeGBPerShard.get(shardNumber - 1) * ReplicaMetricImpl.GB);
             }
             if (leader == null && type != Replica.ReplicaType.PULL) {
               leader = replicaBuilder;
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/BuildersTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/BuildersTest.java
index 941ac1b..5b12b43 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/BuildersTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/BuildersTest.java
@@ -21,6 +21,8 @@ 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.junit.Test;
 
 import java.util.List;
@@ -79,17 +81,17 @@ public class BuildersTest extends SolrTestCaseJ4 {
     AttributeFetcher attributeFetcher = clusterBuilder.buildAttributeFetcher();
     attributeFetcher
         .fetchFrom(cluster.getLiveNodes())
-        .requestNodeCoresCount()
-        .requestNodeFreeDisk()
-        .requestNodeTotalDisk()
-        .requestCollectionMetrics(collection, Set.of(ReplicaMetric.INDEX_SIZE_GB));
+        .requestNodeMetric(NodeMetricImpl.NUM_CORES)
+        .requestNodeMetric(NodeMetricImpl.FREE_DISK_GB)
+        .requestNodeMetric(NodeMetricImpl.TOTAL_DISK_GB)
+        .requestCollectionMetrics(collection, Set.of(ReplicaMetricImpl.INDEX_SIZE_GB));
     AttributeValues attributeValues = attributeFetcher.fetchAttributes();
     for (Node node : cluster.getLiveNodes()) {
-      Optional<Integer> coreCount = attributeValues.getCoresCount(node);
+      Optional<Integer> coreCount = attributeValues.getNodeMetric(node, NodeMetricImpl.NUM_CORES);
       assertTrue("coreCount present", coreCount.isPresent());
-      Optional<Double> diskOpt = attributeValues.getFreeDisk(node);
+      Optional<Double> diskOpt = attributeValues.getNodeMetric(node, NodeMetricImpl.FREE_DISK_GB);
       assertTrue("freeDisk", diskOpt.isPresent());
-      diskOpt = attributeValues.getTotalDisk(node);
+      diskOpt = attributeValues.getNodeMetric(node, NodeMetricImpl.TOTAL_DISK_GB);
       assertTrue("totalDisk", diskOpt.isPresent());
     }
     Optional<CollectionMetrics> collectionMetricsOpt = attributeValues.getCollectionMetrics(collectionName);
@@ -102,7 +104,7 @@ public class BuildersTest extends SolrTestCaseJ4 {
       Optional<ReplicaMetrics> replicaMetricsOpt = shardMetrics.getLeaderMetrics();
       assertTrue("leader metrics", replicaMetricsOpt.isPresent());
       ReplicaMetrics leaderMetrics = replicaMetricsOpt.get();
-      Optional<Double> sizeOpt = leaderMetrics.getReplicaMetric(ReplicaMetric.INDEX_SIZE_GB);
+      Optional<Double> sizeOpt = leaderMetrics.getReplicaMetric(ReplicaMetricImpl.INDEX_SIZE_GB);
       assertTrue("missing size", sizeOpt.isPresent());
       if (shardName.endsWith("1")) {
         assertEquals("size", 10, ((Number) sizeOpt.get()).intValue());
@@ -114,7 +116,7 @@ public class BuildersTest extends SolrTestCaseJ4 {
         Optional<ReplicaMetrics> metricsOpt = shardMetrics.getReplicaMetrics(r.getReplicaName());
         assertTrue("replica metrics", metricsOpt.isPresent());
         ReplicaMetrics metrics = metricsOpt.get();
-        Optional<Double> replicaSizeOpt = metrics.getReplicaMetric(ReplicaMetric.INDEX_SIZE_GB);
+        Optional<Double> replicaSizeOpt = metrics.getReplicaMetric(ReplicaMetricImpl.INDEX_SIZE_GB);
         assertTrue("missing size", replicaSizeOpt.isPresent());
         if (shardName.endsWith("1")) {
           assertEquals("size", 10, ((Number) replicaSizeOpt.get()).intValue());
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 cffe1db..6967550 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
@@ -33,7 +33,6 @@ import org.apache.solr.cluster.placement.CollectionMetrics;
 import org.apache.solr.cluster.placement.NodeMetric;
 import org.apache.solr.cluster.placement.PlacementPluginConfig;
 import org.apache.solr.cluster.placement.PlacementPluginFactory;
-import org.apache.solr.cluster.placement.ReplicaMetric;
 import org.apache.solr.cluster.placement.ReplicaMetrics;
 import org.apache.solr.cluster.placement.ShardMetrics;
 import org.apache.solr.cluster.placement.plugins.AffinityPlacementConfig;
@@ -241,49 +240,41 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
     Cluster cluster = new SimpleClusterAbstractionsImpl.ClusterImpl(cloudManager);
     SolrCollection collection = cluster.getCollection(COLLECTION);
     AttributeFetcher attributeFetcher = new AttributeFetcherImpl(cloudManager);
-    String someMetricKey = "solr.jvm:system.properties:user.name";
+    NodeMetric<String> someMetricKey = new NodeMetricImpl<>("solr.jvm:system.properties:user.name");
     String sysprop = "user.name";
     String sysenv = "PWD";
     attributeFetcher
         .fetchFrom(cluster.getLiveNodes())
-        .requestNodeHeapUsage()
-        .requestNodeSystemLoadAverage()
+        .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(someMetricKey)
-        .requestNodeMetric(NodeMetric.FREE_DISK_GB)
-        .requestNodeMetric(NodeMetric.TOTAL_DISK_GB)
-        // XXX this doesn't work yet, it overwrites
-        // the inserter in AttributeFetcherImpl for nodeCoresCount
-        // .requestNodeMetric(NodeMetric.NUM_CORES)
-        .requestNodeMetric(NodeMetric.SYSLOAD_AVG)
-        .requestNodeMetric(NodeMetric.AVAILABLE_PROCESSORS)
         .requestNodeSystemProperty(sysprop)
         .requestNodeEnvironmentVariable(sysenv)
-        .requestNodeTotalDisk()
-        .requestNodeFreeDisk()
-        .requestNodeCoresCount()
-        .requestCollectionMetrics(collection, Set.of(ReplicaMetric.INDEX_SIZE_GB, ReplicaMetric.QUERY_RATE_1MIN, ReplicaMetric.UPDATE_RATE_1MIN));
+        .requestCollectionMetrics(collection, Set.of(ReplicaMetricImpl.INDEX_SIZE_GB, ReplicaMetricImpl.QUERY_RATE_1MIN, ReplicaMetricImpl.UPDATE_RATE_1MIN));
     AttributeValues attributeValues = attributeFetcher.fetchAttributes();
     String userName = System.getProperty("user.name");
     String pwd = System.getenv("PWD");
     // node metrics
     for (Node node : cluster.getLiveNodes()) {
-      assertTrue("heap usage", attributeValues.getHeapUsage(node).isPresent());
-      assertTrue("total disk 1", attributeValues.getTotalDisk(node).isPresent());
-      Optional<Double> spaceOpt = attributeValues.getNodeMetric(node, NodeMetric.TOTAL_DISK_GB);
-      assertTrue("total disk 2", spaceOpt.isPresent());
-      assertEquals("total disk 1 vs 2", attributeValues.getTotalDisk(node).get(), spaceOpt.get(), 0.0);
-      assertTrue("free disk 1", attributeValues.getFreeDisk(node).isPresent());
-      spaceOpt = attributeValues.getNodeMetric(node, NodeMetric.FREE_DISK_GB);
-      assertTrue("free disk 2", spaceOpt.isPresent());
-      assertEquals("free disk 1 vs 2", attributeValues.getFreeDisk(node).get(), spaceOpt.get(), 0.0);
-      assertTrue("cores count 1", attributeValues.getCoresCount(node).isPresent());
-//      Optional<Integer> coresOpt = attributeValues.getNodeMetric(node, NodeMetric.NUM_CORES);
-//      assertTrue("cores 2", coresOpt.isPresent());
-//      assertEquals("cores 1 vs 2", attributeValues.getCoresCount(node).get(), coresOpt.get());
-      assertTrue("systemLoadAverage 1", attributeValues.getSystemLoadAverage(node).isPresent());
-      assertTrue("systemLoadAverage 2", attributeValues.getNodeMetric(node, NodeMetric.SYSLOAD_AVG).isPresent());
-      assertTrue("availableProcessors", attributeValues.getNodeMetric(node, NodeMetric.AVAILABLE_PROCESSORS).isPresent());
-      Optional<Object> userNameOpt = attributeValues.getNodeMetric(node, someMetricKey);
+      Optional<Double> doubleOpt = attributeValues.getNodeMetric(node, NodeMetricImpl.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);
+      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);
+      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);
+      assertTrue("cores", intOpt.isPresent());
+      assertTrue("cores should be > 0", intOpt.get() > 0);
+      assertTrue("systemLoadAverage 2", attributeValues.getNodeMetric(node, NodeMetricImpl.SYSLOAD_AVG).isPresent());
+      assertTrue("availableProcessors", attributeValues.getNodeMetric(node, NodeMetricImpl.AVAILABLE_PROCESSORS).isPresent());
+      Optional<String> userNameOpt = attributeValues.getNodeMetric(node, someMetricKey);
       assertTrue("user.name", userNameOpt.isPresent());
       assertEquals("userName", userName, userNameOpt.get());
       Optional<String> syspropOpt = attributeValues.getSystemProperty(node, sysprop);
@@ -302,14 +293,14 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
         Optional<ReplicaMetrics> replicaMetricsOpt = shardMetricsOpt.get().getReplicaMetrics(replica.getReplicaName());
         assertTrue("replica metrics", replicaMetricsOpt.isPresent());
         ReplicaMetrics replicaMetrics = replicaMetricsOpt.get();
-        Optional<Double> indexSizeOpt = replicaMetrics.getReplicaMetric(ReplicaMetric.INDEX_SIZE_GB);
+        Optional<Double> indexSizeOpt = replicaMetrics.getReplicaMetric(ReplicaMetricImpl.INDEX_SIZE_GB);
         assertTrue("indexSize", indexSizeOpt.isPresent());
         assertTrue("wrong type, expected Double but was " + indexSizeOpt.get().getClass(), indexSizeOpt.get() instanceof Double);
         assertTrue("indexSize should be > 0 but was " + indexSizeOpt.get(), indexSizeOpt.get() > 0);
         assertTrue("indexSize should be < 0.01 but was " + indexSizeOpt.get(), indexSizeOpt.get() < 0.01);
 
-        assertNotNull("queryRate", replicaMetrics.getReplicaMetric(ReplicaMetric.QUERY_RATE_1MIN));
-        assertNotNull("updateRate", replicaMetrics.getReplicaMetric(ReplicaMetric.UPDATE_RATE_1MIN));
+        assertNotNull("queryRate", replicaMetrics.getReplicaMetric(ReplicaMetricImpl.QUERY_RATE_1MIN));
+        assertNotNull("updateRate", replicaMetrics.getReplicaMetric(ReplicaMetricImpl.UPDATE_RATE_1MIN));
       });
     });
   }