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/10 11:39:39 UTC

[lucene-solr] 02/02: SOLR-15019: Add CollectionMetricsBuilder.

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

commit 19cb4d053a08fc1d6101e6b76c59a560be269246
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Dec 10 12:38:53 2020 +0100

    SOLR-15019: Add CollectionMetricsBuilder.
---
 .../placement/impl/AttributeValuesImpl.java        |  16 ++-
 .../placement/impl/CollectionMetricsBuilder.java   | 128 +++++++++++++++++++++
 .../apache/solr/cluster/placement/Builders.java    |  14 ++-
 3 files changed, 154 insertions(+), 4 deletions(-)

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 4f8b694..c747af3 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
@@ -32,8 +32,12 @@ public class AttributeValuesImpl implements AttributeValues {
   final Map<Node, Long> nodeToTotalDisk;
   final Map<Node, Double> nodeToHeapUsage;
   final Map<Node, Double> nodeToSystemLoadAverage;
+  // sysprop name / node -> value
   final Map<String, Map<Node, String>> syspropSnitchToNodeToValue;
+  // metricName / node -> value
   final Map<String, Map<Node, Double>> metricSnitchToNodeToValue;
+  // collection / shard / replica / metricName -> value
+  final Map<String, CollectionMetrics> collectionMetrics;
 
   public AttributeValuesImpl(Map<Node, Integer> nodeToCoreCount,
                              Map<Node, AttributeFetcher.DiskHardwareType> nodeToDiskType,
@@ -42,7 +46,8 @@ public class AttributeValuesImpl implements AttributeValues {
                              Map<Node, Double> nodeToHeapUsage,
                              Map<Node, Double> nodeToSystemLoadAverage,
                              Map<String, Map<Node, String>> syspropSnitchToNodeToValue,
-                             Map<String, Map<Node, Double>> metricSnitchToNodeToValue) {
+                             Map<String, Map<Node, Double>> metricSnitchToNodeToValue,
+                             Map<String, CollectionMetrics> collectionMetrics) {
     this.nodeToCoreCount = nodeToCoreCount;
     this.nodeToDiskType = nodeToDiskType;
     this.nodeToFreeDisk = nodeToFreeDisk;
@@ -51,6 +56,7 @@ public class AttributeValuesImpl implements AttributeValues {
     this.nodeToSystemLoadAverage = nodeToSystemLoadAverage;
     this.syspropSnitchToNodeToValue = syspropSnitchToNodeToValue;
     this.metricSnitchToNodeToValue = metricSnitchToNodeToValue;
+    this.collectionMetrics = collectionMetrics;
   }
 
   @Override
@@ -115,7 +121,11 @@ public class AttributeValuesImpl implements AttributeValues {
 
   @Override
   public Optional<CollectionMetrics> getCollectionMetrics(String collectionName) {
-    // TODO implement
-    return Optional.empty();
+    CollectionMetrics metrics = collectionMetrics.get(collectionName);
+    if (metrics == null) {
+      return Optional.empty();
+    } else {
+      return Optional.of(metrics);
+    }
   }
 }
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
new file mode 100644
index 0000000..eb29aa8
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/CollectionMetricsBuilder.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cluster.placement.impl;
+
+import org.apache.solr.cluster.placement.CollectionMetrics;
+import org.apache.solr.cluster.placement.ReplicaMetrics;
+import org.apache.solr.cluster.placement.ShardMetrics;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ *
+ */
+public class CollectionMetricsBuilder {
+
+  final Map<String, ShardMetricsBuilder> shardMetricsBuilders = new HashMap<>();
+
+  public void addShard(String shardName, ShardMetricsBuilder shardMetricsBuilder) {
+    shardMetricsBuilders.put(shardName, shardMetricsBuilder);
+  }
+
+  public CollectionMetrics build() {
+    final Map<String, ShardMetrics> metricsMap = new HashMap<>();
+    shardMetricsBuilders.forEach((shard, builder) -> metricsMap.put(shard, builder.build()));
+    return shardName -> {
+      if (metricsMap.containsKey(shardName)) {
+        return Optional.of(metricsMap.get(shardName));
+      } else {
+        return Optional.empty();
+      }
+    };
+  }
+
+  public static class ShardMetricsBuilder {
+    final Map<String, ReplicaMetricsBuilder> replicaMetricsBuilders = new HashMap<>();
+
+    public void addReplica(String replicaName, ReplicaMetricsBuilder replicaMetricsBuilder) {
+      replicaMetricsBuilders.put(replicaName, replicaMetricsBuilder);
+    }
+
+    private static final String LEADER = "__leader__";
+
+    public ShardMetrics build() {
+      final Map<String, ReplicaMetrics> metricsMap = new HashMap<>();
+      replicaMetricsBuilders.forEach((name, replicaBuilder) -> {
+        ReplicaMetrics metrics = replicaBuilder.build();
+        metricsMap.put(name, metrics);
+        if (replicaBuilder.leader) {
+          metricsMap.put(LEADER, metrics);
+        }
+      });
+      return new ShardMetrics() {
+        @Override
+        public Optional<ReplicaMetrics> getLeaderMetrics() {
+          if (metricsMap.containsKey(LEADER)) {
+            return Optional.of(metricsMap.get(LEADER));
+          } else {
+            return Optional.empty();
+          }
+        }
+
+        @Override
+        public Optional<ReplicaMetrics> getReplicaMetrics(String replicaName) {
+          if (metricsMap.containsKey(replicaName)) {
+            return Optional.of(metricsMap.get(replicaName));
+          } else {
+            return Optional.empty();
+          }
+        }
+      };
+    }
+  }
+
+  public static class ReplicaMetricsBuilder {
+    final Map<String, Object> metrics = new HashMap<>();
+    int sizeGB = 0;
+    boolean leader;
+
+    public ReplicaMetricsBuilder setSizeGB(int size) {
+      this.sizeGB = size;
+      return this;
+    }
+
+    public ReplicaMetricsBuilder setLeader(boolean leader) {
+      this.leader = leader;
+      return this;
+    }
+
+    public ReplicaMetricsBuilder addMetric(String metricName, Object value) {
+      metrics.put(metricName, value);
+      return this;
+    }
+
+    public ReplicaMetrics build() {
+      return new ReplicaMetrics() {
+        @Override
+        public int getReplicaSizeGB() {
+          return sizeGB;
+        }
+
+        @Override
+        public Optional<Object> getReplicaMetric(String metricName) {
+          if (metrics.containsKey(metricName)) {
+            return Optional.of(metrics.get(metricName));
+          } else {
+            return Optional.empty();
+          }
+        }
+      };
+    }
+  }
+}
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 398b16d..9350a71 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
@@ -20,6 +20,7 @@ package org.apache.solr.cluster.placement;
 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.common.util.Pair;
 import org.junit.Assert;
 
@@ -91,6 +92,7 @@ public class Builders {
       Map<Node, Long> nodeToFreeDisk = new HashMap<>();
       Map<String, Map<Node, String>> sysprops = new HashMap<>();
       Map<String, Map<Node, Double>> metrics = new HashMap<>();
+      Map<String, CollectionMetrics> collectionMetrics = new HashMap<>();
 
       // TODO And a few more missing and will be added...
 
@@ -119,7 +121,12 @@ public class Builders {
         }
       }
 
-      AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), sysprops, metrics);
+      collectionBuilders.forEach(builder -> {
+        collectionMetrics.put(builder.collectionName, builder.collectionMetricsBuilder.build());
+      });
+
+      AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk,
+          Map.of(), Map.of(), Map.of(), sysprops, metrics, collectionMetrics);
       return new AttributeFetcherForTest(attributeValues);
     }
   }
@@ -129,6 +136,7 @@ public class Builders {
     private LinkedList<ShardBuilder> shardBuilders = new LinkedList<>();
     private Map<String, String> customProperties = new HashMap<>();
     int replicaNumber = 0; // global replica numbering for the collection
+    private CollectionMetricsBuilder collectionMetricsBuilder = new CollectionMetricsBuilder();
 
     public CollectionBuilder(String collectionName) {
       this.collectionName = collectionName;
@@ -139,6 +147,10 @@ public class Builders {
       return this;
     }
 
+    public CollectionMetricsBuilder getCollectionMetricsBuilder() {
+      return collectionMetricsBuilder;
+    }
+
     /**
      * @return The internal shards data structure to allow test code to modify the replica distribution to nodes.
      */