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/16 13:17:39 UTC

[lucene-solr] branch jira/solr-15019 updated: SOLR-15019: Add support for sysprops and sysenv.

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 61facaa  SOLR-15019: Add support for sysprops and sysenv.
61facaa is described below

commit 61facaa065bb7026b3ba10ebf01d03df207f7235
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Wed Dec 16 14:17:09 2020 +0100

    SOLR-15019: Add support for sysprops and sysenv.
---
 .../placement/impl/AttributeFetcherImpl.java       | 19 ++++++++-----
 .../placement/impl/AttributeValuesImpl.java        | 24 ++++++++--------
 .../apache/solr/servlet/SolrDispatchFilter.java    |  8 ++++++
 .../impl/PlacementPluginIntegrationTest.java       | 11 ++++++++
 .../solrj/impl/SolrClientNodeStateProvider.java    | 33 +++++++++++++---------
 .../solr/common/cloud/rule/ImplicitSnitch.java     |  1 +
 6 files changed, 63 insertions(+), 33 deletions(-)

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 f998468..9101c97 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
@@ -46,7 +46,7 @@ public class AttributeFetcherImpl implements AttributeFetcher {
   boolean requestedNodeTotalDisk;
   boolean requestedNodeHeapUsage;
   boolean requestedNodeSystemLoadAverage;
-  Set<String> requestedNodeSystemPropertiesSnitchTags = new HashSet<>();
+  Set<String> requestedNodeSystemSnitchTags = new HashSet<>();
   Set<String> requestedNodeMetricSnitchTags = new HashSet<>();
   Map<SolrCollection, Set<ReplicaMetric>> requestedCollectionMetrics = new HashMap<>();
 
@@ -95,13 +95,14 @@ public class AttributeFetcherImpl implements AttributeFetcher {
 
   @Override
   public AttributeFetcher requestNodeSystemProperty(String name) {
-    requestedNodeSystemPropertiesSnitchTags.add(getSystemPropertySnitchTag(name));
+    requestedNodeSystemSnitchTags.add(getSystemPropertySnitchTag(name));
     return this;
   }
 
   @Override
   public AttributeFetcher requestNodeEnvironmentVariable(String name) {
-    throw new UnsupportedOperationException("Not yet implemented...");
+    requestedNodeSystemSnitchTags.add(getSystemEnvSnitchTag(name));
+    return this;
   }
 
   @Override
@@ -141,7 +142,7 @@ public class AttributeFetcherImpl implements AttributeFetcher {
     Map<Node, Double> nodeToTotalDisk = new HashMap<>();
     Map<Node, Double> nodeToHeapUsage = new HashMap<>();
     Map<Node, Double> nodeToSystemLoadAverage = new HashMap<>();
-    Map<String, Map<Node, String>> syspropSnitchToNodeToValue = new HashMap<>();
+    Map<String, Map<Node, String>> systemSnitchToNodeToValue = new HashMap<>();
     Map<String, Map<Node, Object>> metricSnitchToNodeToValue = new HashMap<>();
     Map<String, CollectionMetricsBuilder> collectionMetricsBuilders = new HashMap<>();
     Map<Node, Set<String>> nodeToReplicaInternalTags = new HashMap<>();
@@ -175,9 +176,9 @@ public class AttributeFetcherImpl implements AttributeFetcher {
       allSnitchTagsToInsertion.put(ImplicitSnitch.SYSLOADAVG,
           (node, value) -> nodeToSystemLoadAverage.put(node, ((Number) value).doubleValue()));
     }
-    for (String sysPropSnitch : requestedNodeSystemPropertiesSnitchTags) {
+    for (String sysPropSnitch : requestedNodeSystemSnitchTags) {
       final Map<Node, String> sysPropMap = new HashMap<>();
-      syspropSnitchToNodeToValue.put(sysPropSnitch, sysPropMap);
+      systemSnitchToNodeToValue.put(sysPropSnitch, sysPropMap);
       allSnitchTagsToInsertion.put(sysPropSnitch, (node, value) -> sysPropMap.put(node, (String) value));
     }
     for (String metricSnitch : requestedNodeMetricSnitchTags) {
@@ -264,7 +265,7 @@ public class AttributeFetcherImpl implements AttributeFetcher {
         nodeToTotalDisk,
         nodeToHeapUsage,
         nodeToSystemLoadAverage,
-        syspropSnitchToNodeToValue,
+        systemSnitchToNodeToValue,
         metricSnitchToNodeToValue, collectionMetrics);
   }
 
@@ -290,4 +291,8 @@ public class AttributeFetcherImpl implements AttributeFetcher {
   public static String getSystemPropertySnitchTag(String name) {
     return ImplicitSnitch.SYSPROP + name;
   }
+
+  public static String getSystemEnvSnitchTag(String name) {
+    return ImplicitSnitch.SYSENV + name;
+  }
 }
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 7714d88..898d272 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,8 @@ public class AttributeValuesImpl implements AttributeValues {
   final Map<Node, Double> nodeToTotalDisk;
   final Map<Node, Double> nodeToHeapUsage;
   final Map<Node, Double> nodeToSystemLoadAverage;
-  // sysprop name / node -> value
-  final Map<String, Map<Node, String>> syspropSnitchToNodeToValue;
+  // sysprop (or sysenv) name / node -> value
+  final Map<String, Map<Node, String>> systemSnitchToNodeToValue;
   // metricName / node -> value
   final Map<String, Map<Node, Object>> metricSnitchToNodeToValue;
   // collection / shard / replica / metricName -> value
@@ -45,7 +45,7 @@ public class AttributeValuesImpl implements AttributeValues {
                              Map<Node, Double> nodeToTotalDisk,
                              Map<Node, Double> nodeToHeapUsage,
                              Map<Node, Double> nodeToSystemLoadAverage,
-                             Map<String, Map<Node, String>> syspropSnitchToNodeToValue,
+                             Map<String, Map<Node, String>> systemSnitchToNodeToValue,
                              Map<String, Map<Node, Object>> metricSnitchToNodeToValue,
                              Map<String, CollectionMetrics> collectionMetrics) {
     this.nodeToCoreCount = nodeToCoreCount;
@@ -54,7 +54,7 @@ public class AttributeValuesImpl implements AttributeValues {
     this.nodeToTotalDisk = nodeToTotalDisk;
     this.nodeToHeapUsage = nodeToHeapUsage;
     this.nodeToSystemLoadAverage = nodeToSystemLoadAverage;
-    this.syspropSnitchToNodeToValue = syspropSnitchToNodeToValue;
+    this.systemSnitchToNodeToValue = systemSnitchToNodeToValue;
     this.metricSnitchToNodeToValue = metricSnitchToNodeToValue;
     this.collectionMetrics = collectionMetrics;
   }
@@ -91,7 +91,7 @@ public class AttributeValuesImpl implements AttributeValues {
 
   @Override
   public Optional<String> getSystemProperty(Node node, String name) {
-    Map<Node, String> nodeToValue = syspropSnitchToNodeToValue.get(AttributeFetcherImpl.getSystemPropertySnitchTag(name));
+    Map<Node, String> nodeToValue = systemSnitchToNodeToValue.get(AttributeFetcherImpl.getSystemPropertySnitchTag(name));
     if (nodeToValue == null) {
       return Optional.empty();
     }
@@ -100,8 +100,11 @@ public class AttributeValuesImpl implements AttributeValues {
 
   @Override
   public Optional<String> getEnvironmentVariable(Node node, String name) {
-    // TODO implement
-    return Optional.empty();
+    Map<Node, String> nodeToValue = systemSnitchToNodeToValue.get(AttributeFetcherImpl.getSystemEnvSnitchTag(name));
+    if (nodeToValue == null) {
+      return Optional.empty();
+    }
+    return Optional.ofNullable(nodeToValue.get(node));
   }
 
   @Override
@@ -124,11 +127,6 @@ public class AttributeValuesImpl implements AttributeValues {
 
   @Override
   public Optional<CollectionMetrics> getCollectionMetrics(String collectionName) {
-    CollectionMetrics metrics = collectionMetrics.get(collectionName);
-    if (metrics == null) {
-      return Optional.empty();
-    } else {
-      return Optional.of(metrics);
-    }
+    return Optional.ofNullable(collectionMetrics.get(collectionName));
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 0c983f1..3a294c5 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -244,6 +244,14 @@ public class SolrDispatchFilter extends BaseSolrFilter {
         });
       });
       metricManager.registerGauge(null, registryName, sysprops, metricTag, SolrMetricManager.ResolutionStrategy.IGNORE, "properties", "system");
+      MetricsMap sysenv = new MetricsMap(map -> {
+        System.getenv().forEach((k, v) -> {
+          if (!hiddenSysProps.contains(k)) {
+            map.putNoEx(String.valueOf(k), v);
+          }
+        });
+      });
+      metricManager.registerGauge(null, registryName, sysenv, metricTag, SolrMetricManager.ResolutionStrategy.IGNORE, "env", "system");
     } catch (Exception e) {
       log.warn("Error registering JVM metrics", e);
     }
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 1b86dff..88fb673 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
@@ -241,16 +241,21 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
     SolrCollection collection = cluster.getCollection(COLLECTION);
     AttributeFetcher attributeFetcher = new AttributeFetcherImpl(cloudManager);
     String someMetricName = "solr.jvm:system.properties:user.name";
+    String sysprop = "user.name";
+    String sysenv = "PWD";
     attributeFetcher
         .fetchFrom(cluster.getLiveNodes())
         .requestNodeHeapUsage()
         .requestNodeMetric(someMetricName)
+        .requestNodeSystemProperty(sysprop)
+        .requestNodeEnvironmentVariable(sysenv)
         .requestNodeTotalDisk()
         .requestNodeFreeDisk()
         .requestNodeCoresCount()
         .requestCollectionMetrics(collection, Set.of(ReplicaMetric.QUERY_RATE_1MIN, ReplicaMetric.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());
@@ -260,6 +265,12 @@ public class PlacementPluginIntegrationTest extends SolrCloudTestCase {
       Optional<Object> userNameOpt = attributeValues.getNodeMetric(node, someMetricName);
       assertTrue("user.name", userNameOpt.isPresent());
       assertEquals("userName", userName, userNameOpt.get());
+      Optional<String> syspropOpt = attributeValues.getSystemProperty(node, sysprop);
+      assertTrue("sysprop", syspropOpt.isPresent());
+      assertEquals("user.name sysprop", userName, syspropOpt.get());
+      Optional<String> sysenvOpt = attributeValues.getEnvironmentVariable(node, sysenv);
+      assertTrue("sysenv", sysenvOpt.isPresent());
+      assertEquals("PWD sysenv", pwd, sysenvOpt.get());
     }
     assertTrue(attributeValues.getCollectionMetrics(COLLECTION).isPresent());
     CollectionMetrics collectionMetrics = attributeValues.getCollectionMetrics(COLLECTION).get();
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
index 9f7e9e8..a7e43c3 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
@@ -162,28 +162,30 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
   }
 
   protected Map<String, Object> fetchReplicaMetrics(String node, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
-    Map<String, Object> collect = metricsKeyVsTagReplica.entrySet().stream()
-            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getKey));
+    Map<String, Set<Object>> collect = metricsKeyVsTagReplica.entrySet().stream()
+            .collect(Collectors.toMap(e -> e.getKey(), e -> Set.of(e.getKey())));
     ClientSnitchCtx ctx = new ClientSnitchCtx(null, null, emptyMap(), solrClient);
     fetchReplicaMetrics(node, ctx, collect);
     return ctx.getTags();
 
   }
 
-  static void fetchReplicaMetrics(String solrNode, ClientSnitchCtx ctx, Map<String, Object> metricsKeyVsTag) {
+  static void fetchReplicaMetrics(String solrNode, ClientSnitchCtx ctx, Map<String, Set<Object>> metricsKeyVsTag) {
     if (!ctx.isNodeAlive(solrNode)) return;
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.add("key", metricsKeyVsTag.keySet().toArray(new String[0]));
     try {
       SimpleSolrResponse rsp = ctx.invokeWithRetry(solrNode, CommonParams.METRICS_PATH, params);
-      metricsKeyVsTag.forEach((key, tag) -> {
+      metricsKeyVsTag.forEach((key, tags) -> {
         Object v = Utils.getObjectByPath(rsp.nl, true, Arrays.asList("metrics", key));
-        if (tag instanceof Function) {
-          @SuppressWarnings({"unchecked"})
-          Pair<String, Object> p = (Pair<String, Object>) ((Function) tag).apply(v);
-          ctx.getTags().put(p.first(), p.second());
-        } else {
-          if (v != null) ctx.getTags().put(tag.toString(), v);
+        for (Object tag : tags) {
+          if (tag instanceof Function) {
+            @SuppressWarnings({"unchecked"})
+            Pair<String, Object> p = (Pair<String, Object>) ((Function) tag).apply(v);
+            ctx.getTags().put(p.first(), p.second());
+          } else {
+            if (v != null) ctx.getTags().put(tag.toString(), v);
+          }
         }
       });
     } catch (Exception e) {
@@ -202,12 +204,17 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
     protected void getRemoteInfo(String solrNode, Set<String> requestedTags, SnitchContext ctx) {
       if (!((ClientSnitchCtx)ctx).isNodeAlive(solrNode)) return;
       ClientSnitchCtx snitchContext = (ClientSnitchCtx) ctx;
-      Map<String, Object> metricsKeyVsTag = new HashMap<>();
+      Map<String, Set<Object>> metricsKeyVsTag = new HashMap<>();
       for (String tag : requestedTags) {
         if (tag.startsWith(SYSPROP)) {
-          metricsKeyVsTag.put("solr.jvm:system.properties:" + tag.substring(SYSPROP.length()), tag);
+          metricsKeyVsTag.computeIfAbsent("solr.jvm:system.properties:" + tag.substring(SYSPROP.length()), k -> new HashSet<>())
+              .add(tag);
+        } else if (tag.startsWith(SYSENV)) {
+            metricsKeyVsTag.computeIfAbsent("solr.jvm:system.env:" + tag.substring(SYSENV.length()), k -> new HashSet<>())
+              .add(tag);
         } else if (tag.startsWith(METRICS_PREFIX)) {
-          metricsKeyVsTag.put(tag.substring(METRICS_PREFIX.length()), tag);
+          metricsKeyVsTag.computeIfAbsent(tag.substring(METRICS_PREFIX.length()), k -> new HashSet<>())
+              .add(tag);
         }
       }
       if (!metricsKeyVsTag.isEmpty()) {
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 1c861b5..c3be955 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
@@ -49,6 +49,7 @@ public class ImplicitSnitch extends Snitch {
   public static final String ROLE = "role";
   public static final String NODEROLE = "nodeRole";
   public static final String SYSPROP = "sysprop.";
+  public static final String SYSENV = "sysenv.";
   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"));