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 12:52:37 UTC

[lucene-solr] branch jira/solr-15019 updated: SOLR-15019: Refactor AttributeFetcher / AttributeValues to use NodeMetric-s where possible.

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 f028414  SOLR-15019: Refactor AttributeFetcher / AttributeValues to use NodeMetric-s where possible.
f028414 is described below

commit f0284141e50813d82ed70848168ddf7043ca3592
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue Dec 22 13:51:50 2020 +0100

    SOLR-15019: Refactor AttributeFetcher / AttributeValues to use NodeMetric-s where possible.
---
 .../solr/cluster/placement/AttributeFetcher.java   | 25 +++++---
 .../solr/cluster/placement/AttributeValues.java    | 24 +++++--
 .../apache/solr/cluster/placement/NodeMetric.java  |  6 +-
 .../placement/impl/AttributeFetcherImpl.java       | 74 +---------------------
 .../placement/impl/AttributeValuesImpl.java        | 42 +-----------
 .../cluster/placement/AttributeFetcherForTest.java |  5 --
 .../apache/solr/cluster/placement/Builders.java    | 34 +++++-----
 .../solr/cluster/placement/BuildersTest.java       |  1 -
 .../solr/common/cloud/rule/ImplicitSnitch.java     |  2 +-
 9 files changed, 59 insertions(+), 154 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/AttributeFetcher.java b/solr/core/src/java/org/apache/solr/cluster/placement/AttributeFetcher.java
index f7e7908..aaa6377 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
@@ -29,32 +29,37 @@ public interface AttributeFetcher {
   /**
    * Request the number of cores on each node. To get the value use {@link AttributeValues#getCoresCount(Node)}
    */
-  AttributeFetcher requestNodeCoresCount();
-
-  /**
-   * Request the disk hardware type on each node. To get the value use {@link AttributeValues#getDiskType(Node)}
-   */
-  AttributeFetcher requestNodeDiskType();
+  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)}
    */
-  AttributeFetcher requestNodeFreeDisk();
+  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)}
    */
-  AttributeFetcher requestNodeTotalDisk();
+  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)}
    */
-  AttributeFetcher requestNodeHeapUsage();
+  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)}
    */
-  AttributeFetcher requestNodeSystemLoadAverage();
+  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)}
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 88400b8..9fe0a1b 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
@@ -25,27 +25,37 @@ public interface AttributeValues {
   /**
    * For the given node: number of cores
    */
-  Optional<Integer> getCoresCount(Node node);
+  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
    */
-  Optional<Double> getFreeDisk(Node node);
+  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
    */
-  Optional<Double> getTotalDisk(Node node);
+  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
    */
-  Optional<Double> getHeapUsage(Node node);
+  default Optional<Double> getHeapUsage(Node node) {
+    return getNodeMetric(node, NodeMetric.HEAP_USAGE);
+  }
 
   /**
    * For the given node: matches {@link java.lang.management.OperatingSystemMXBean#getSystemLoadAverage()}
    */
-  Optional<Double> getSystemLoadAverage(Node node);
+  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}
@@ -66,7 +76,9 @@ public interface AttributeValues {
   /**
    * Get any metric using a fully-qualified metric key.
    */
-  Optional<Object> getNodeMetric(Node node, String metricKey);
+  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/NodeMetric.java b/solr/core/src/java/org/apache/solr/cluster/placement/NodeMetric.java
index 12ac0d5..d4f382e 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,6 +17,8 @@
 
 package org.apache.solr.cluster.placement;
 
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+
 import java.util.Objects;
 import java.util.function.Function;
 
@@ -34,7 +36,9 @@ public class NodeMetric<T> extends MetricAttribute<T> {
   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);
+  /** 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 =
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 3a5770e..a5837a5 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
@@ -60,41 +60,6 @@ public class AttributeFetcherImpl implements AttributeFetcher {
   }
 
   @Override
-  public AttributeFetcher requestNodeCoresCount() {
-    requestedNodeCoreCount = true;
-    return this;
-  }
-
-  @Override
-  public AttributeFetcher requestNodeDiskType() {
-    throw new UnsupportedOperationException("Not yet implemented...");
-  }
-
-  @Override
-  public AttributeFetcher requestNodeFreeDisk() {
-    requestedNodeFreeDisk = true;
-    return this;
-  }
-
-  @Override
-  public AttributeFetcher requestNodeTotalDisk() {
-    requestedNodeTotalDisk = true;
-    return this;
-  }
-
-  @Override
-  public AttributeFetcher requestNodeHeapUsage() {
-    requestedNodeHeapUsage = true;
-    return this;
-  }
-
-  @Override
-  public AttributeFetcher requestNodeSystemLoadAverage() {
-    requestedNodeSystemLoadAverage = true;
-    return this;
-  }
-
-  @Override
   public AttributeFetcher requestNodeSystemProperty(String name) {
     requestedNodeSystemSnitchTags.add(getSystemPropertySnitchTag(name));
     return this;
@@ -113,12 +78,6 @@ public class AttributeFetcherImpl implements AttributeFetcher {
   }
 
   @Override
-  public AttributeFetcher requestNodeMetric(String metricKey) {
-    requestedNodeMetricSnitchTags.add(new NodeMetric<>(metricKey));
-    return this;
-  }
-
-  @Override
   public AttributeFetcher requestCollectionMetrics(SolrCollection solrCollection, Set<ReplicaMetric<?>> metrics) {
     if (!metrics.isEmpty()) {
       requestedCollectionMetrics.put(solrCollection, Set.copyOf(metrics));
@@ -139,11 +98,6 @@ public class AttributeFetcherImpl implements AttributeFetcher {
     // TODO Code here only supports node related attributes for now
 
     // Maps in which attribute values will be added
-    Map<Node, Integer> nodeToCoreCount = new HashMap<>();
-    Map<Node, Double> nodeToFreeDisk = new HashMap<>();
-    Map<Node, Double> nodeToTotalDisk = new HashMap<>();
-    Map<Node, Double> nodeToHeapUsage = new HashMap<>();
-    Map<Node, Double> nodeToSystemLoadAverage = new HashMap<>();
     Map<String, Map<Node, String>> systemSnitchToNodeToValue = new HashMap<>();
     Map<NodeMetric<?>, Map<Node, Object>> metricSnitchToNodeToValue = new HashMap<>();
     Map<String, CollectionMetricsBuilder> collectionMetricsBuilders = new HashMap<>();
@@ -157,27 +111,6 @@ public class AttributeFetcherImpl implements AttributeFetcher {
     // 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<>();
-    if (requestedNodeCoreCount) {
-      allSnitchTagsToInsertion.put(ImplicitSnitch.CORES, (node, value) -> nodeToCoreCount.put(node, ((Number) value).intValue()));
-    }
-    if (requestedNodeFreeDisk) {
-      allSnitchTagsToInsertion.put(SolrClientNodeStateProvider.Variable.FREEDISK.tagName,
-          // Convert from bytes to GB
-          (node, value) -> nodeToFreeDisk.put(node, ((Number) value).doubleValue() / GB));
-    }
-    if (requestedNodeTotalDisk) {
-      allSnitchTagsToInsertion.put(SolrClientNodeStateProvider.Variable.TOTALDISK.tagName,
-          // Convert from bytes to GB
-          (node, value) -> nodeToTotalDisk.put(node, ((Number) value).doubleValue() / GB));
-    }
-    if (requestedNodeHeapUsage) {
-      allSnitchTagsToInsertion.put(ImplicitSnitch.HEAPUSAGE,
-          (node, value) -> nodeToHeapUsage.put(node, ((Number) value).doubleValue()));
-    }
-    if (requestedNodeSystemLoadAverage) {
-      allSnitchTagsToInsertion.put(ImplicitSnitch.SYSLOADAVG,
-          (node, value) -> nodeToSystemLoadAverage.put(node, ((Number) value).doubleValue()));
-    }
     for (String sysPropSnitch : requestedNodeSystemSnitchTags) {
       final Map<Node, String> sysPropMap = new HashMap<>();
       systemSnitchToNodeToValue.put(sysPropSnitch, sysPropMap);
@@ -252,12 +185,7 @@ public class AttributeFetcherImpl implements AttributeFetcher {
     Map<String, CollectionMetrics> collectionMetrics = new HashMap<>();
     collectionMetricsBuilders.forEach((name, builder) -> collectionMetrics.put(name, builder.build()));
 
-    return new AttributeValuesImpl(nodeToCoreCount,
-        nodeToFreeDisk,
-        nodeToTotalDisk,
-        nodeToHeapUsage,
-        nodeToSystemLoadAverage,
-        systemSnitchToNodeToValue,
+    return new AttributeValuesImpl(systemSnitchToNodeToValue,
         metricSnitchToNodeToValue, collectionMetrics);
   }
 
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 f16fcf8..6101e3d 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
@@ -26,11 +26,6 @@ import java.util.Map;
 import java.util.Optional;
 
 public class AttributeValuesImpl implements AttributeValues {
-  final Map<Node, Integer> nodeToCoreCount;
-  final Map<Node, Double> nodeToFreeDisk;
-  final Map<Node, Double> nodeToTotalDisk;
-  final Map<Node, Double> nodeToHeapUsage;
-  final Map<Node, Double> nodeToSystemLoadAverage;
   // sysprop (or sysenv) name / node -> value
   final Map<String, Map<Node, String>> systemSnitchToNodeToValue;
   // metricName / node -> value
@@ -38,50 +33,15 @@ public class AttributeValuesImpl implements AttributeValues {
   // collection / shard / replica / metricName -> value
   final Map<String, CollectionMetrics> collectionMetrics;
 
-  public AttributeValuesImpl(Map<Node, Integer> nodeToCoreCount,
-                             Map<Node, Double> nodeToFreeDisk,
-                             Map<Node, Double> nodeToTotalDisk,
-                             Map<Node, Double> nodeToHeapUsage,
-                             Map<Node, Double> nodeToSystemLoadAverage,
-                             Map<String, Map<Node, String>> systemSnitchToNodeToValue,
+  public AttributeValuesImpl(Map<String, Map<Node, String>> systemSnitchToNodeToValue,
                              Map<NodeMetric<?>, Map<Node, Object>> metricSnitchToNodeToValue,
                              Map<String, CollectionMetrics> collectionMetrics) {
-    this.nodeToCoreCount = nodeToCoreCount;
-    this.nodeToFreeDisk = nodeToFreeDisk;
-    this.nodeToTotalDisk = nodeToTotalDisk;
-    this.nodeToHeapUsage = nodeToHeapUsage;
-    this.nodeToSystemLoadAverage = nodeToSystemLoadAverage;
     this.systemSnitchToNodeToValue = systemSnitchToNodeToValue;
     this.metricSnitchToNodeToValue = metricSnitchToNodeToValue;
     this.collectionMetrics = collectionMetrics;
   }
 
   @Override
-  public Optional<Integer> getCoresCount(Node node) {
-    return Optional.ofNullable(nodeToCoreCount.get(node));
-  }
-
-  @Override
-  public Optional<Double> getFreeDisk(Node node) {
-    return Optional.ofNullable(nodeToFreeDisk.get(node));
-  }
-
-  @Override
-  public Optional<Double> getTotalDisk(Node node) {
-    return Optional.ofNullable(nodeToTotalDisk.get(node));
-  }
-
-  @Override
-  public Optional<Double> getHeapUsage(Node node) {
-    return Optional.ofNullable(nodeToHeapUsage.get(node));
-  }
-
-  @Override
-  public Optional<Double> getSystemLoadAverage(Node node) {
-    return Optional.ofNullable(nodeToSystemLoadAverage.get(node));
-  }
-
-  @Override
   public Optional<String> getSystemProperty(Node node, String name) {
     Map<Node, String> nodeToValue = systemSnitchToNodeToValue.get(AttributeFetcherImpl.getSystemPropertySnitchTag(name));
     if (nodeToValue == null) {
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 004f8e2..e7903b9 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
@@ -36,11 +36,6 @@ public class AttributeFetcherForTest implements AttributeFetcher {
   }
 
   @Override
-  public AttributeFetcher requestNodeDiskType() {
-    return this;
-  }
-
-  @Override
   public AttributeFetcher requestNodeFreeDisk() {
     return this;
   }
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 e024798..2afad19 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
@@ -91,9 +91,6 @@ public class Builders {
     }
 
     public AttributeFetcher buildAttributeFetcher() {
-      Map<Node, Integer> nodeToCoreCount = new HashMap<>();
-      Map<Node, Double> nodeToFreeDisk = new HashMap<>();
-      Map<Node, Double> nodeToTotalDisk = new HashMap<>();
       Map<String, Map<Node, String>> sysprops = new HashMap<>();
       Map<NodeMetric<?>, Map<Node, Object>> metrics = new HashMap<>();
       Map<String, CollectionMetrics> collectionMetrics = new HashMap<>();
@@ -106,13 +103,16 @@ public class Builders {
         Node node = nodeBuilder.build();
 
         if (nodeBuilder.getCoreCount() != null) {
-          nodeToCoreCount.put(node, nodeBuilder.getCoreCount());
+          metrics.computeIfAbsent(NodeMetric.NUM_CORES, n -> new HashMap<>())
+              .put(node, nodeBuilder.getCoreCount());
         }
         if (nodeBuilder.getFreeDiskGB() != null) {
-          nodeToFreeDisk.put(node, nodeBuilder.getFreeDiskGB());
+          metrics.computeIfAbsent(NodeMetric.FREE_DISK_GB, n -> new HashMap<>())
+              .put(node, nodeBuilder.getFreeDiskGB());
         }
         if (nodeBuilder.getTotalDiskGB() != null) {
-          nodeToTotalDisk.put(node, nodeBuilder.getTotalDiskGB());
+          metrics.computeIfAbsent(NodeMetric.TOTAL_DISK_GB, n -> new HashMap<>())
+              .put(node, nodeBuilder.getTotalDiskGB());
         }
         if (nodeBuilder.getSysprops() != null) {
           nodeBuilder.getSysprops().forEach((name, value) -> {
@@ -128,18 +128,20 @@ public class Builders {
         }
       }
 
-      collectionBuilders.forEach(builder -> {
-        collectionMetrics.put(builder.collectionName, builder.collectionMetricsBuilder.build());
-        SolrCollection collection = builder.build();
-        collection.iterator().forEachRemaining(shard ->
-            shard.iterator().forEachRemaining(replica -> {
+      if (!collectionBuilders.isEmpty()) {
+        Map<Node, Object> nodeToCoreCount = metrics.computeIfAbsent(NodeMetric.NUM_CORES, n -> new HashMap<>());
+        collectionBuilders.forEach(builder -> {
+          collectionMetrics.put(builder.collectionName, builder.collectionMetricsBuilder.build());
+          SolrCollection collection = builder.build();
+          collection.iterator().forEachRemaining(shard ->
+              shard.iterator().forEachRemaining(replica -> {
                 nodeToCoreCount.compute(replica.getNode(), (node, count) ->
-                    (count == null) ? 1 : count + 1);
-            }));
-      });
+                    (count == null) ? 1 : ((Number) count).intValue() + 1);
+              }));
+        });
+      }
 
-      AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, nodeToFreeDisk,
-          nodeToTotalDisk, Map.of(), Map.of(), sysprops, metrics, collectionMetrics);
+      AttributeValues attributeValues = new AttributeValuesImpl(sysprops, metrics, collectionMetrics);
       return new AttributeFetcherForTest(attributeValues);
     }
   }
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 561c8ec..941ac1b 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
@@ -80,7 +80,6 @@ public class BuildersTest extends SolrTestCaseJ4 {
     attributeFetcher
         .fetchFrom(cluster.getLiveNodes())
         .requestNodeCoresCount()
-        .requestNodeDiskType()
         .requestNodeFreeDisk()
         .requestNodeTotalDisk()
         .requestCollectionMetrics(collection, Set.of(ReplicaMetric.INDEX_SIZE_GB));
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/rule/ImplicitSnitch.java b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/ImplicitSnitch.java
index c3be955..acc412f 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/rule/ImplicitSnitch.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/ImplicitSnitch.java
@@ -53,7 +53,7 @@ public class ImplicitSnitch extends Snitch {
   public static final String SYSLOADAVG = "sysLoadAvg";
   public static final String HEAPUSAGE = "heapUsage";
   public static final List<String> IP_SNITCHES = Collections.unmodifiableList(Arrays.asList("ip_1", "ip_2", "ip_3", "ip_4"));
-  public static final Set<String> tags = Set.of(NODE, PORT, HOST, CORES, DISK, ROLE, "ip_1", "ip_2", "ip_3", "ip_4");
+  public static final Set<String> tags = Set.of(NODE, PORT, HOST, CORES, DISK, ROLE, HEAPUSAGE, "ip_1", "ip_2", "ip_3", "ip_4");
 
   @Override
   public void getTags(String solrNode, Set<String> requestedTags, SnitchContext ctx) {