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 11:48:43 UTC

[lucene-solr] branch jira/solr-15019 updated: SOLR-15019: Add a couple common node metrics.

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 6c7b95c  SOLR-15019: Add a couple common node metrics.
6c7b95c is described below

commit 6c7b95caab8e71710e0fce3f5e42df86e2d8b56b
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue Dec 22 12:48:08 2020 +0100

    SOLR-15019: Add a couple common node metrics.
---
 .../solr/cluster/placement/CollectionMetrics.java  |  4 +-
 .../solr/cluster/placement/MetricAttribute.java    | 58 ++++++++++++++++++++--
 .../apache/solr/cluster/placement/NodeMetric.java  | 18 +++++--
 .../solr/cluster/placement/ReplicaMetric.java      | 23 ++-------
 .../solr/cluster/placement/ReplicaMetrics.java     |  2 +-
 .../solr/cluster/placement/ShardMetrics.java       |  4 +-
 .../placement/impl/AttributeFetcherImpl.java       |  5 ++
 .../impl/PlacementPluginIntegrationTest.java       | 20 ++++++--
 8 files changed, 100 insertions(+), 34 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/CollectionMetrics.java b/solr/core/src/java/org/apache/solr/cluster/placement/CollectionMetrics.java
index 8336881..5093745 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/CollectionMetrics.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/CollectionMetrics.java
@@ -20,7 +20,9 @@ package org.apache.solr.cluster.placement;
 import java.util.Optional;
 
 /**
- *
+ * Collection-level metrics. Currently this class is a container just
+ * for shard-level metrics but future versions may add other
+ * primitive collection-level metrics.
  */
 public interface CollectionMetrics {
 
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 f0a8a15..a638660 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
@@ -20,16 +20,21 @@ import java.util.Objects;
 import java.util.function.Function;
 
 /**
- * Replica metric wrapper that defines a short symbolic name of the metric, the corresponding
- * internal metric name (as reported in <code>solr.core.[collection].[replica]</code> registry)
- * and the desired format/unit conversion.
+ * 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.
  */
 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")
-  private final Function<Object, T> IDENTITY_CONVERTER = v -> {
+  public final Function<Object, T> IDENTITY_CONVERTER = v -> {
     try {
       return (T) v;
     } catch (ClassCastException cce) {
@@ -37,14 +42,48 @@ public class 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.
+   */
   public MetricAttribute(String name, String internalName) {
     this(name, internalName, null);
   }
 
+  /**
+   * 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);
@@ -57,14 +96,25 @@ public class MetricAttribute<T> {
     }
   }
 
+  /**
+   * Return the short-hand name that identifies this attribute.
+   */
   public String getName() {
     return name;
   }
 
+  /**
+   * Return the internal name of a Solr metric associated with this attribute.
+   */
   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);
   }
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 3aea6d2..12ac0d5 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
@@ -21,16 +21,26 @@ import java.util.Objects;
 import java.util.function.Function;
 
 /**
- * Node metric wrapper that defines a short symbolic name of the metric, the corresponding
- * metric registry and the internal metric name, as well as the desired format/unit conversion.
+ * Node metric identifier, corresponding
+ * to a node-level metric registry and the internal metric name.
  */
 public class NodeMetric<T> extends MetricAttribute<T> {
 
-  /** System load average metric. */
+  /** 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);
+
+  // public static final NodeMetric<Integer> NUM_CORES = new NodeMetric<>(ImplicitSnitch.CORES);
+
+  /** System load average. */
   public static final NodeMetric<Double> SYSLOAD_AVG =
       new NodeMetric<>("sysLoadAvg", AttributeFetcher.NodeMetricRegistry.SOLR_JVM, "os.systemLoadAverage");
 
-  /** Available processors metric. */
+  /** Number of available processors. */
   public static final NodeMetric<Integer> AVAILABLE_PROCESSORS =
       new NodeMetric<>("availableProcessors", AttributeFetcher.NodeMetricRegistry.SOLR_JVM, "os.availableProcessors");
 
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 98bc5d4..8d9c25d 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
@@ -19,29 +19,12 @@ package org.apache.solr.cluster.placement;
 import java.util.function.Function;
 
 /**
- * Replica metric wrapper that defines a short symbolic name of the metric, the corresponding
- * internal metric name (as reported in <code>solr.core.[collection].[replica]</code> registry)
- * and the desired format/unit conversion.
+ * 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",
-      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;
-      });
+  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");
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/ReplicaMetrics.java b/solr/core/src/java/org/apache/solr/cluster/placement/ReplicaMetrics.java
index bab16b2..f5783e8 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/ReplicaMetrics.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/ReplicaMetrics.java
@@ -20,7 +20,7 @@ package org.apache.solr.cluster.placement;
 import java.util.Optional;
 
 /**
- *
+ * Strongly-typed replica-level metrics.
  */
 public interface ReplicaMetrics {
 
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/ShardMetrics.java b/solr/core/src/java/org/apache/solr/cluster/placement/ShardMetrics.java
index 3f63316..79d5966 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/ShardMetrics.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/ShardMetrics.java
@@ -19,7 +19,9 @@ package org.apache.solr.cluster.placement;
 import java.util.Optional;
 
 /**
- *
+ * Shard-level metrics. Currently this is just a container for
+ * replica-level metrics but future versions may add other
+ * primitive shard-level metrics.
  */
 public interface ShardMetrics {
   Optional<ReplicaMetrics> getLeaderMetrics();
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 bcba85c..3a5770e 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
@@ -274,9 +274,14 @@ public class AttributeFetcherImpl implements AttributeFetcher {
 
   public static String getMetricSnitchTag(NodeMetric<?> metric) {
     if (metric.getRegistry() != null) {
+      // regular registry + metricName
       return SolrClientNodeStateProvider.METRICS_PREFIX +
           SolrMetricManager.getRegistryName(getGroupFromMetricRegistry(metric.getRegistry())) + ":" + metric.getInternalName();
+    } else if (ImplicitSnitch.tags.contains(metric.getInternalName())) {
+      // "special" well-known tag
+      return metric.getInternalName();
     } else {
+      // a fully-qualified metric key
       return SolrClientNodeStateProvider.METRICS_PREFIX + metric.getInternalName();
     }
   }
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 eb80b18..cffe1db 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
@@ -249,6 +249,11 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
         .requestNodeHeapUsage()
         .requestNodeSystemLoadAverage()
         .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)
@@ -263,9 +268,18 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
     // node metrics
     for (Node node : cluster.getLiveNodes()) {
       assertTrue("heap usage", attributeValues.getHeapUsage(node).isPresent());
-      assertTrue("total disk", attributeValues.getTotalDisk(node).isPresent());
-      assertTrue("free disk", attributeValues.getFreeDisk(node).isPresent());
-      assertTrue("cores count", attributeValues.getCoresCount(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());