Browse Source

Scripting: stats per context in nodes stats (#54008)

Adds script cache stats to `_node/stats`.
If using the general cache:
```
      "script_cache": {
        "sum": {
          "compilations": 12,
          "cache_evictions": 9,
          "compilation_limit_triggered": 5
        }
      }

```
If using context caches:
```
      "script_cache": {       
        "sum": {                 
          "compilations": 13,                
          "cache_evictions": 9,
          "compilation_limit_triggered": 5
        },                                
        "contexts": [         
          {                      
            "context": "aggregation_selector",
            "compilations": 8,
            "cache_evictions": 6,
            "compilation_limit_triggered": 3
          },                  
          {                      
            "context": "aggs",              
            "compilations": 5,
            "cache_evictions": 3,
            "compilation_limit_triggered": 2
          },                  
```
Refs: #50152
Stuart Tettemer 5 years ago
parent
commit
32f46f2214
18 changed files with 338 additions and 31 deletions
  1. 24 1
      server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java
  2. 2 1
      server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodesStatsRequest.java
  3. 8 0
      server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodesStatsRequestBuilder.java
  4. 2 1
      server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java
  5. 1 1
      server/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java
  6. 3 2
      server/src/main/java/org/elasticsearch/node/NodeService.java
  7. 5 2
      server/src/main/java/org/elasticsearch/script/ScriptCache.java
  8. 147 0
      server/src/main/java/org/elasticsearch/script/ScriptCacheStats.java
  9. 19 2
      server/src/main/java/org/elasticsearch/script/ScriptService.java
  10. 42 3
      server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java
  11. 12 6
      server/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java
  12. 8 6
      server/src/test/java/org/elasticsearch/script/ScriptCacheTests.java
  13. 60 1
      server/src/test/java/org/elasticsearch/script/ScriptServiceTests.java
  14. 1 1
      test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java
  15. 1 1
      test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java
  16. 1 1
      x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MachineLearningInfoTransportActionTests.java
  17. 1 1
      x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportGetTrainedModelsStatsActionTests.java
  18. 1 1
      x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/node/NodeStatsMonitoringDocTests.java

+ 24 - 1
server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java

@@ -19,6 +19,7 @@
 
 package org.elasticsearch.action.admin.cluster.node.stats;
 
+import org.elasticsearch.Version;
 import org.elasticsearch.action.support.nodes.BaseNodeResponse;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodeRole;
@@ -37,6 +38,7 @@ import org.elasticsearch.monitor.jvm.JvmStats;
 import org.elasticsearch.monitor.os.OsStats;
 import org.elasticsearch.monitor.process.ProcessStats;
 import org.elasticsearch.node.AdaptiveSelectionStats;
+import org.elasticsearch.script.ScriptCacheStats;
 import org.elasticsearch.script.ScriptStats;
 import org.elasticsearch.threadpool.ThreadPoolStats;
 import org.elasticsearch.transport.TransportStats;
@@ -81,6 +83,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
     @Nullable
     private ScriptStats scriptStats;
 
+    @Nullable
+    private ScriptCacheStats scriptCacheStats;
+
     @Nullable
     private DiscoveryStats discoveryStats;
 
@@ -108,6 +113,11 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
         discoveryStats = in.readOptionalWriteable(DiscoveryStats::new);
         ingestStats = in.readOptionalWriteable(IngestStats::new);
         adaptiveSelectionStats = in.readOptionalWriteable(AdaptiveSelectionStats::new);
+        if (in.getVersion().onOrAfter(Version.V_7_8_0)) {
+            scriptCacheStats = in.readOptionalWriteable(ScriptCacheStats::new);
+        } else {
+            scriptCacheStats = null;
+        }
     }
 
     public NodeStats(DiscoveryNode node, long timestamp, @Nullable NodeIndicesStats indices,
@@ -117,7 +127,8 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
                      @Nullable ScriptStats scriptStats,
                      @Nullable DiscoveryStats discoveryStats,
                      @Nullable IngestStats ingestStats,
-                     @Nullable AdaptiveSelectionStats adaptiveSelectionStats) {
+                     @Nullable AdaptiveSelectionStats adaptiveSelectionStats,
+                     @Nullable ScriptCacheStats scriptCacheStats) {
         super(node);
         this.timestamp = timestamp;
         this.indices = indices;
@@ -133,6 +144,7 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
         this.discoveryStats = discoveryStats;
         this.ingestStats = ingestStats;
         this.adaptiveSelectionStats = adaptiveSelectionStats;
+        this.scriptCacheStats = scriptCacheStats;
     }
 
     public long getTimestamp() {
@@ -227,6 +239,11 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
         return adaptiveSelectionStats;
     }
 
+    @Nullable
+    public ScriptCacheStats getScriptCacheStats() {
+        return scriptCacheStats;
+    }
+
     @Override
     public void writeTo(StreamOutput out) throws IOException {
         super.writeTo(out);
@@ -249,6 +266,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
         out.writeOptionalWriteable(discoveryStats);
         out.writeOptionalWriteable(ingestStats);
         out.writeOptionalWriteable(adaptiveSelectionStats);
+        if (out.getVersion().onOrAfter(Version.V_7_8_0)) {
+            out.writeOptionalWriteable(scriptCacheStats);
+        }
     }
 
     @Override
@@ -312,6 +332,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
         if (getAdaptiveSelectionStats() != null) {
             getAdaptiveSelectionStats().toXContent(builder, params);
         }
+        if (getScriptCacheStats() != null) {
+            getScriptCacheStats().toXContent(builder, params);
+        }
         return builder;
     }
 }

+ 2 - 1
server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodesStatsRequest.java

@@ -219,7 +219,8 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
         SCRIPT("script"),
         DISCOVERY("discovery"),
         INGEST("ingest"),
-        ADAPTIVE_SELECTION("adaptive_selection");
+        ADAPTIVE_SELECTION("adaptive_selection"),
+        SCRIPT_CACHE("script_cache");
 
         private String metricName;
 

+ 8 - 0
server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodesStatsRequestBuilder.java

@@ -149,6 +149,14 @@ public class NodesStatsRequestBuilder
         return this;
     }
 
+    /**
+     * Should script context cache statistics be returned
+     */
+    public NodesStatsRequestBuilder setScriptCache(boolean scriptCache) {
+        addOrRemoveMetric(scriptCache, NodesStatsRequest.Metric.SCRIPT_CACHE);
+        return this;
+    }
+
     /**
      * Helper method for adding metrics to a request
      */

+ 2 - 1
server/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java

@@ -83,7 +83,8 @@ public class TransportNodesStatsAction extends TransportNodesAction<NodesStatsRe
             NodesStatsRequest.Metric.SCRIPT.containedIn(metrics),
             NodesStatsRequest.Metric.DISCOVERY.containedIn(metrics),
             NodesStatsRequest.Metric.INGEST.containedIn(metrics),
-            NodesStatsRequest.Metric.ADAPTIVE_SELECTION.containedIn(metrics));
+            NodesStatsRequest.Metric.ADAPTIVE_SELECTION.containedIn(metrics),
+            NodesStatsRequest.Metric.SCRIPT_CACHE.containedIn(metrics));
     }
 
     public static class NodeStatsRequest extends BaseNodeRequest {

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java

@@ -98,7 +98,7 @@ public class TransportClusterStatsAction extends TransportNodesAction<ClusterSta
     protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeRequest, Task task) {
         NodeInfo nodeInfo = nodeService.info(true, true, false, true, false, true, false, true, false, false);
         NodeStats nodeStats = nodeService.stats(CommonStatsFlags.NONE,
-                true, true, true, false, true, false, false, false, false, false, true, false);
+                true, true, true, false, true, false, false, false, false, false, true, false, false);
         List<ShardStats> shardsStats = new ArrayList<>();
         for (IndexService indexService : indicesService) {
             for (IndexShard indexShard : indexService) {

+ 3 - 2
server/src/main/java/org/elasticsearch/node/NodeService.java

@@ -103,7 +103,7 @@ public class NodeService implements Closeable {
 
     public NodeStats stats(CommonStatsFlags indices, boolean os, boolean process, boolean jvm, boolean threadPool,
                            boolean fs, boolean transport, boolean http, boolean circuitBreaker,
-                           boolean script, boolean discoveryStats, boolean ingest, boolean adaptiveSelection) {
+                           boolean script, boolean discoveryStats, boolean ingest, boolean adaptiveSelection, boolean scriptCache) {
         // for indices stats we want to include previous allocated shards stats as well (it will
         // only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats)
         return new NodeStats(transportService.getLocalNode(), System.currentTimeMillis(),
@@ -119,7 +119,8 @@ public class NodeService implements Closeable {
                 script ? scriptService.stats() : null,
                 discoveryStats ? discovery.stats() : null,
                 ingest ? ingestService.stats() : null,
-                adaptiveSelection ? responseCollectorService.getAdaptiveStats(searchTransportService.getPendingSearchRequests()) : null
+                adaptiveSelection ? responseCollectorService.getAdaptiveStats(searchTransportService.getPendingSearchRequests()) : null,
+                scriptCache ? scriptService.cacheStats() : null
         );
     }
 

+ 5 - 2
server/src/main/java/org/elasticsearch/script/ScriptCache.java

@@ -56,14 +56,17 @@ public class ScriptCache {
     final TimeValue cacheExpire;
     final Tuple<Integer, TimeValue> rate;
     private final double compilesAllowedPerNano;
+    private final String contextRateSetting;
 
     ScriptCache(
             int cacheMaxSize,
             TimeValue cacheExpire,
-            Tuple<Integer, TimeValue> maxCompilationRate
+            Tuple<Integer, TimeValue> maxCompilationRate,
+            String contextRateSetting
     ) {
         this.cacheSize = cacheMaxSize;
         this.cacheExpire = cacheExpire;
+        this.contextRateSetting = contextRateSetting;
 
         CacheBuilder<CacheKey, Object> cacheBuilder = CacheBuilder.builder();
         if (this.cacheSize >= 0) {
@@ -175,7 +178,7 @@ public class ScriptCache {
             // Otherwise reject the request
             throw new CircuitBreakingException("[script] Too many dynamic script compilations within, max: [" +
                 rate.v1() + "/" + rate.v2() +"]; please use indexed, or scripts with parameters instead; " +
-                "this limit can be changed by the [script.max_compilations_rate] setting",
+                "this limit can be changed by the [" + contextRateSetting + "] setting",
                 CircuitBreaker.Durability.TRANSIENT);
         }
     }

+ 147 - 0
server/src/main/java/org/elasticsearch/script/ScriptCacheStats.java

@@ -0,0 +1,147 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch 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.elasticsearch.script;
+
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.xcontent.ToXContentFragment;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+public class ScriptCacheStats implements Writeable, ToXContentFragment {
+    private final Map<String,ScriptStats> context;
+    private final ScriptStats general;
+
+    public ScriptCacheStats(Map<String, ScriptStats> context) {
+        this.context = Collections.unmodifiableMap(context);
+        this.general = null;
+    }
+
+    public ScriptCacheStats(ScriptStats general) {
+        this.general = Objects.requireNonNull(general);
+        this.context = null;
+    }
+
+    public ScriptCacheStats(StreamInput in) throws IOException {
+        boolean isContext = in.readBoolean();
+        if (isContext == false) {
+            general = new ScriptStats(in);
+            context = null;
+            return;
+        }
+
+        general = null;
+        int size = in.readInt();
+        Map<String,ScriptStats> context = new HashMap<>(size);
+        for (int i=0; i < size; i++) {
+            String name = in.readString();
+            context.put(name, new ScriptStats(in));
+        }
+        this.context = Collections.unmodifiableMap(context);
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        if (general != null) {
+            out.writeBoolean(false);
+            general.writeTo(out);
+            return;
+        }
+
+        out.writeBoolean(true);
+        out.writeInt(context.size());
+        for (String name: context.keySet().stream().sorted().collect(Collectors.toList())) {
+            out.writeString(name);
+            context.get(name).writeTo(out);
+        }
+    }
+
+    @Override
+    public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+        builder.startObject(Fields.SCRIPT_CACHE_STATS);
+        builder.startObject(Fields.SUM);
+        if (general != null) {
+            builder.field(ScriptStats.Fields.COMPILATIONS, general.getCompilations());
+            builder.field(ScriptStats.Fields.CACHE_EVICTIONS, general.getCacheEvictions());
+            builder.field(ScriptStats.Fields.COMPILATION_LIMIT_TRIGGERED, general.getCompilationLimitTriggered());
+            builder.endObject().endObject();
+            return builder;
+        }
+
+        ScriptStats sum = sum();
+        builder.field(ScriptStats.Fields.COMPILATIONS, sum.getCompilations());
+        builder.field(ScriptStats.Fields.CACHE_EVICTIONS, sum.getCacheEvictions());
+        builder.field(ScriptStats.Fields.COMPILATION_LIMIT_TRIGGERED, sum.getCompilationLimitTriggered());
+        builder.endObject();
+
+        builder.startArray(Fields.CONTEXTS);
+        for (String name: context.keySet().stream().sorted().collect(Collectors.toList())) {
+            ScriptStats stats = context.get(name);
+            builder.startObject();
+            builder.field(Fields.CONTEXT, name);
+            builder.field(ScriptStats.Fields.COMPILATIONS, stats.getCompilations());
+            builder.field(ScriptStats.Fields.CACHE_EVICTIONS, stats.getCacheEvictions());
+            builder.field(ScriptStats.Fields.COMPILATION_LIMIT_TRIGGERED, stats.getCompilationLimitTriggered());
+            builder.endObject();
+        }
+        builder.endArray();
+        builder.endObject();
+
+        return builder;
+    }
+
+    /**
+     * Get the context specific stats, null if using general cache
+     */
+    public Map<String, ScriptStats> getContextStats() {
+        return context;
+    }
+
+    /**
+     * Get the general stats, null if using context cache
+     */
+    public ScriptStats getGeneralStats() {
+        return general;
+    }
+
+    /**
+     * The sum of all script stats, either the general stats or the sum of all stats of the context stats.
+     */
+    public ScriptStats sum() {
+        if (general != null) {
+            return general;
+        }
+        return ScriptStats.sum(context.values());
+    }
+
+    static final class Fields {
+        static final String SCRIPT_CACHE_STATS = "script_cache";
+        static final String CONTEXT = "context";
+        static final String SUM = "sum";
+        static final String CONTEXTS = "contexts";
+    }
+}

+ 19 - 2
server/src/main/java/org/elasticsearch/script/ScriptService.java

@@ -567,6 +567,10 @@ public class ScriptService implements Closeable, ClusterStateApplier {
         return cacheHolder.get().stats();
     }
 
+    public ScriptCacheStats cacheStats() {
+        return cacheHolder.get().cacheStats();
+    }
+
     @Override
     public void applyClusterState(ClusterChangedEvent event) {
         clusterState = event.state();
@@ -602,7 +606,8 @@ public class ScriptService implements Closeable, ClusterStateApplier {
                     SCRIPT_GENERAL_CACHE_EXPIRE_SETTING.get(settings),
                     compilationLimitsEnabled ?
                         SCRIPT_GENERAL_MAX_COMPILATIONS_RATE_SETTING.get(settings) :
-                        SCRIPT_COMPILATION_RATE_ZERO);
+                        SCRIPT_COMPILATION_RATE_ZERO,
+                    SCRIPT_GENERAL_MAX_COMPILATIONS_RATE_SETTING.getKey());
             }
         }
 
@@ -626,7 +631,8 @@ public class ScriptService implements Closeable, ClusterStateApplier {
 
             return new ScriptCache(cacheSize.existsOrFallbackExists(settings) ? cacheSize.get(settings) : context.cacheSizeDefault,
                                    cacheExpire.existsOrFallbackExists(settings) ? cacheExpire.get(settings) : context.cacheExpireDefault,
-                                   compileRate);
+                                   compileRate,
+                                   SCRIPT_MAX_COMPILATIONS_RATE_SETTING.getConcreteSettingForNamespace(context.name).getKey());
         }
 
         /**
@@ -677,6 +683,17 @@ public class ScriptService implements Closeable, ClusterStateApplier {
             return ScriptStats.sum(contextCache.values().stream().map(AtomicReference::get).map(ScriptCache::stats)::iterator);
         }
 
+        ScriptCacheStats cacheStats() {
+            if (general != null) {
+                return new ScriptCacheStats(general.stats());
+            }
+            Map<String, ScriptStats> context = new HashMap<>(contextCache.size());
+            for (ScriptContext<?> ctx: contexts) {
+                context.put(ctx.name, contextCache.get(ctx.name).get().stats());
+            }
+            return new ScriptCacheStats(context);
+        }
+
         /**
          * Update settings for the context cache, if we're in the context cache mode otherwise no-op.
          */

+ 42 - 3
server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java

@@ -19,12 +19,12 @@
 
 package org.elasticsearch.action.admin.cluster.node.stats;
 
+import org.elasticsearch.cluster.coordination.PendingClusterStateStats;
+import org.elasticsearch.cluster.coordination.PublishClusterStateStats;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.discovery.DiscoveryStats;
-import org.elasticsearch.cluster.coordination.PendingClusterStateStats;
-import org.elasticsearch.cluster.coordination.PublishClusterStateStats;
 import org.elasticsearch.http.HttpStats;
 import org.elasticsearch.indices.breaker.AllCircuitBreakerStats;
 import org.elasticsearch.indices.breaker.CircuitBreakerStats;
@@ -35,6 +35,7 @@ import org.elasticsearch.monitor.os.OsStats;
 import org.elasticsearch.monitor.process.ProcessStats;
 import org.elasticsearch.node.AdaptiveSelectionStats;
 import org.elasticsearch.node.ResponseCollectorService;
+import org.elasticsearch.script.ScriptCacheStats;
 import org.elasticsearch.script.ScriptStats;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.test.VersionUtils;
@@ -311,6 +312,34 @@ public class NodeStatsTests extends ESTestCase {
                         assertEquals(aStats.responseTime, bStats.responseTime, 0.01);
                     });
                 }
+                ScriptCacheStats scriptCacheStats = nodeStats.getScriptCacheStats();
+                ScriptCacheStats deserializedScriptCacheStats = deserializedNodeStats.getScriptCacheStats();
+                if (scriptCacheStats == null) {
+                    assertNull(deserializedScriptCacheStats);
+                } else {
+                    Map<String, ScriptStats> deserialized = deserializedScriptCacheStats.getContextStats();
+                    long evictions = 0;
+                    long limited = 0;
+                    long compilations = 0;
+                    Map<String, ScriptStats> stats = scriptCacheStats.getContextStats();
+                    for (String context: stats.keySet()) {
+                        ScriptStats deserStats = deserialized.get(context);
+                        ScriptStats generatedStats = stats.get(context);
+
+                        evictions += generatedStats.getCacheEvictions();
+                        assertEquals(generatedStats.getCacheEvictions(), deserStats.getCacheEvictions());
+
+                        limited += generatedStats.getCompilationLimitTriggered();
+                        assertEquals(generatedStats.getCompilationLimitTriggered(), deserStats.getCompilationLimitTriggered());
+
+                        compilations += generatedStats.getCompilations();
+                        assertEquals(generatedStats.getCompilations(), deserStats.getCompilations());
+                    }
+                    ScriptStats sum = deserializedScriptCacheStats.sum();
+                    assertEquals(evictions, sum.getCacheEvictions());
+                    assertEquals(limited, sum.getCompilationLimitTriggered());
+                    assertEquals(compilations, sum.getCompilations());
+                }
             }
         }
     }
@@ -485,10 +514,20 @@ public class NodeStatsTests extends ESTestCase {
             }
             adaptiveSelectionStats = new AdaptiveSelectionStats(nodeConnections, nodeStats);
         }
+        ScriptCacheStats scriptCacheStats = null;
+        if (frequently()) {
+            int numContents = randomIntBetween(0, 20);
+            Map<String,ScriptStats> stats = new HashMap<>(numContents);
+            for (int i = 0; i < numContents; i++) {
+                String context = randomValueOtherThanMany(stats::containsKey, () -> randomAlphaOfLength(12));
+                stats.put(context, new ScriptStats(randomLongBetween(0, 1024), randomLongBetween(0, 1024), randomLongBetween(0, 1024)));
+            }
+            scriptCacheStats = new ScriptCacheStats(stats);
+        }
         //TODO NodeIndicesStats are not tested here, way too complicated to create, also they need to be migrated to Writeable yet
         return new NodeStats(node, randomNonNegativeLong(), null, osStats, processStats, jvmStats, threadPoolStats,
                 fsInfo, transportStats, httpStats, allCircuitBreakerStats, scriptStats, discoveryStats,
-                ingestStats, adaptiveSelectionStats);
+                ingestStats, adaptiveSelectionStats, scriptCacheStats);
     }
 
     private IngestStats.Stats getPipelineStats(List<IngestStats.PipelineStat> pipelineStats, String id) {

+ 12 - 6
server/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java

@@ -151,11 +151,14 @@ public class DiskUsageTests extends ESTestCase {
         };
         List<NodeStats> nodeStats = Arrays.asList(
                 new NodeStats(new DiscoveryNode("node_1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
-                        null,null,null,null,null,new FsInfo(0, null, node1FSInfo), null,null,null,null,null, null, null),
+                        null,null,null,null,null,new FsInfo(0, null, node1FSInfo), null,null,null,null,null, null, null,
+                        null),
                 new NodeStats(new DiscoveryNode("node_2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
-                        null,null,null,null,null, new FsInfo(0, null, node2FSInfo), null,null,null,null,null, null, null),
+                        null,null,null,null,null, new FsInfo(0, null, node2FSInfo), null,null,null,null,null, null, null,
+                        null),
                 new NodeStats(new DiscoveryNode("node_3", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
-                        null,null,null,null,null, new FsInfo(0, null, node3FSInfo), null,null,null,null,null, null, null)
+                        null,null,null,null,null, new FsInfo(0, null, node3FSInfo), null,null,null,null,null, null, null,
+                        null)
         );
         InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvaiableUsages, newMostAvaiableUsages);
         DiskUsage leastNode_1 = newLeastAvaiableUsages.get("node_1");
@@ -192,11 +195,14 @@ public class DiskUsageTests extends ESTestCase {
         };
         List<NodeStats> nodeStats = Arrays.asList(
                 new NodeStats(new DiscoveryNode("node_1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
-                        null,null,null,null,null,new FsInfo(0, null, node1FSInfo), null,null,null,null,null, null, null),
+                        null,null,null,null,null,new FsInfo(0, null, node1FSInfo), null,null,null,null,null, null, null,
+                        null),
                 new NodeStats(new DiscoveryNode("node_2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
-                        null,null,null,null,null, new FsInfo(0, null, node2FSInfo), null,null,null,null,null, null, null),
+                        null,null,null,null,null, new FsInfo(0, null, node2FSInfo), null,null,null,null,null, null, null,
+                        null),
                 new NodeStats(new DiscoveryNode("node_3", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
-                        null,null,null,null,null, new FsInfo(0, null, node3FSInfo), null,null,null,null,null, null, null)
+                        null,null,null,null,null, new FsInfo(0, null, node3FSInfo), null,null,null,null,null, null, null,
+                        null)
         );
         InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvailableUsages, newMostAvailableUsages);
         DiskUsage leastNode_1 = newLeastAvailableUsages.get("node_1");

+ 8 - 6
server/src/test/java/org/elasticsearch/script/ScriptCacheTests.java

@@ -31,22 +31,23 @@ public class ScriptCacheTests extends ESTestCase {
         final TimeValue expire = ScriptService.SCRIPT_GENERAL_CACHE_EXPIRE_SETTING.get(Settings.EMPTY);
         final Integer size = ScriptService.SCRIPT_GENERAL_CACHE_SIZE_SETTING.get(Settings.EMPTY);
         Tuple<Integer, TimeValue> rate = ScriptService.SCRIPT_GENERAL_MAX_COMPILATIONS_RATE_SETTING.get(Settings.EMPTY);
-        ScriptCache cache = new ScriptCache(size, expire, Tuple.tuple(1, TimeValue.timeValueMinutes(1)));
+        String settingName = ScriptService.SCRIPT_GENERAL_MAX_COMPILATIONS_RATE_SETTING.getKey();
+        ScriptCache cache = new ScriptCache(size, expire, Tuple.tuple(1, TimeValue.timeValueMinutes(1)), settingName);
         cache.checkCompilationLimit(); // should pass
         expectThrows(CircuitBreakingException.class, cache::checkCompilationLimit);
-        cache = new ScriptCache(size, expire, (Tuple.tuple(2, TimeValue.timeValueMinutes(1))));
+        cache = new ScriptCache(size, expire, (Tuple.tuple(2, TimeValue.timeValueMinutes(1))), settingName);
         cache.checkCompilationLimit(); // should pass
         cache.checkCompilationLimit(); // should pass
         expectThrows(CircuitBreakingException.class, cache::checkCompilationLimit);
         int count = randomIntBetween(5, 50);
-        cache = new ScriptCache(size, expire, (Tuple.tuple(count, TimeValue.timeValueMinutes(1))));
+        cache = new ScriptCache(size, expire, (Tuple.tuple(count, TimeValue.timeValueMinutes(1))), settingName);
         for (int i = 0; i < count; i++) {
             cache.checkCompilationLimit(); // should pass
         }
         expectThrows(CircuitBreakingException.class, cache::checkCompilationLimit);
-        cache = new ScriptCache(size, expire, (Tuple.tuple(0, TimeValue.timeValueMinutes(1))));
+        cache = new ScriptCache(size, expire, (Tuple.tuple(0, TimeValue.timeValueMinutes(1))), settingName);
         expectThrows(CircuitBreakingException.class, cache::checkCompilationLimit);
-        cache = new ScriptCache(size, expire, (Tuple.tuple(Integer.MAX_VALUE, TimeValue.timeValueMinutes(1))));
+        cache = new ScriptCache(size, expire, (Tuple.tuple(Integer.MAX_VALUE, TimeValue.timeValueMinutes(1))), settingName);
         int largeLimit = randomIntBetween(1000, 10000);
         for (int i = 0; i < largeLimit; i++) {
             cache.checkCompilationLimit();
@@ -56,7 +57,8 @@ public class ScriptCacheTests extends ESTestCase {
     public void testUnlimitedCompilationRate() {
         final Integer size = ScriptService.SCRIPT_GENERAL_CACHE_SIZE_SETTING.get(Settings.EMPTY);
         final TimeValue expire = ScriptService.SCRIPT_GENERAL_CACHE_EXPIRE_SETTING.get(Settings.EMPTY);
-        ScriptCache cache = new ScriptCache(size, expire, ScriptCache.UNLIMITED_COMPILATION_RATE);
+        String settingName = ScriptService.SCRIPT_GENERAL_MAX_COMPILATIONS_RATE_SETTING.getKey();
+        ScriptCache cache = new ScriptCache(size, expire, ScriptCache.UNLIMITED_COMPILATION_RATE, settingName);
         long lastInlineCompileTime = cache.lastInlineCompileTime;
         double scriptsPerTimeWindow = cache.scriptsPerTimeWindow;
         for(int i=0; i < 3000; i++) {

+ 60 - 1
server/src/test/java/org/elasticsearch/script/ScriptServiceTests.java

@@ -23,6 +23,7 @@ import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptReque
 import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.MetaData;
+import org.elasticsearch.common.breaker.CircuitBreakingException;
 import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.collect.Tuple;
@@ -43,6 +44,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
@@ -221,7 +223,6 @@ public class ScriptServiceTests extends ESTestCase {
         scriptService.compile(new Script(ScriptType.INLINE, "test", "1+1", Collections.emptyMap()), randomFrom(contexts.values()));
         assertEquals(1L, scriptService.stats().getCompilations());
     }
-
     public void testMultipleCompilationsCountedInCompilationStats() throws IOException {
         buildScriptService(Settings.EMPTY);
         int numberOfCompilations = randomIntBetween(1, 20);
@@ -247,6 +248,7 @@ public class ScriptServiceTests extends ESTestCase {
         buildScriptService(Settings.EMPTY);
         scriptService.compile(new Script(ScriptType.STORED, null, "script", Collections.emptyMap()), randomFrom(contexts.values()));
         assertEquals(1L, scriptService.stats().getCompilations());
+        assertEquals(1L, scriptService.cacheStats().getGeneralStats().getCompilations());
     }
 
     public void testCacheEvictionCountedInCacheEvictionsStats() throws IOException {
@@ -256,7 +258,64 @@ public class ScriptServiceTests extends ESTestCase {
         scriptService.compile(new Script(ScriptType.INLINE, "test", "1+1", Collections.emptyMap()), randomFrom(contexts.values()));
         scriptService.compile(new Script(ScriptType.INLINE, "test", "2+2", Collections.emptyMap()), randomFrom(contexts.values()));
         assertEquals(2L, scriptService.stats().getCompilations());
+        assertEquals(2L, scriptService.cacheStats().getGeneralStats().getCompilations());
         assertEquals(1L, scriptService.stats().getCacheEvictions());
+        assertEquals(1L, scriptService.cacheStats().getGeneralStats().getCacheEvictions());
+    }
+
+    public void testContextCacheStats() throws IOException {
+        ScriptContext<?> contextA = randomFrom(contexts.values());
+        String aRate = "2/10m";
+        ScriptContext<?> contextB = randomFrom(contexts.values());
+        String bRate = "3/10m";
+        BiFunction<String, String, String> msg = (rate, ctx) -> (
+            "[script] Too many dynamic script compilations within, max: [" + rate +
+            "]; please use indexed, or scripts with parameters instead; this limit can be changed by the [script.context." + ctx +
+            ".max_compilations_rate] setting"
+        );
+        buildScriptService(Settings.builder()
+            .put(SCRIPT_GENERAL_MAX_COMPILATIONS_RATE_SETTING.getKey(), USE_CONTEXT_RATE_KEY)
+            .put(SCRIPT_CACHE_SIZE_SETTING.getConcreteSettingForNamespace(contextA.name).getKey(), 1)
+            .put(SCRIPT_MAX_COMPILATIONS_RATE_SETTING.getConcreteSettingForNamespace(contextA.name).getKey(), "2/10m")
+            .put(SCRIPT_CACHE_SIZE_SETTING.getConcreteSettingForNamespace(contextB.name).getKey(), 2)
+            .put(SCRIPT_MAX_COMPILATIONS_RATE_SETTING.getConcreteSettingForNamespace(contextB.name).getKey(), "3/10m")
+            .build());
+
+        // Context A
+        scriptService.compile(new Script(ScriptType.INLINE, "test", "1+1", Collections.emptyMap()), contextA);
+        scriptService.compile(new Script(ScriptType.INLINE, "test", "2+2", Collections.emptyMap()), contextA);
+        GeneralScriptException gse = expectThrows(GeneralScriptException.class,
+            () -> scriptService.compile(new Script(ScriptType.INLINE, "test", "3+3", Collections.emptyMap()), contextA));
+        assertEquals(msg.apply(aRate, contextA.name), gse.getRootCause().getMessage());
+        assertEquals(CircuitBreakingException.class, gse.getRootCause().getClass());
+
+        // Context B
+        scriptService.compile(new Script(ScriptType.INLINE, "test", "4+4", Collections.emptyMap()), contextB);
+        scriptService.compile(new Script(ScriptType.INLINE, "test", "5+5", Collections.emptyMap()), contextB);
+        scriptService.compile(new Script(ScriptType.INLINE, "test", "6+6", Collections.emptyMap()), contextB);
+        gse = expectThrows(GeneralScriptException.class,
+            () -> scriptService.compile(new Script(ScriptType.INLINE, "test", "7+7", Collections.emptyMap()), contextB));
+        assertEquals(msg.apply(bRate, contextB.name), gse.getRootCause().getMessage());
+        gse = expectThrows(GeneralScriptException.class,
+            () -> scriptService.compile(new Script(ScriptType.INLINE, "test", "8+8", Collections.emptyMap()), contextB));
+        assertEquals(msg.apply(bRate, contextB.name), gse.getRootCause().getMessage());
+        assertEquals(CircuitBreakingException.class, gse.getRootCause().getClass());
+
+        // Context specific
+        ScriptCacheStats stats = scriptService.cacheStats();
+        assertEquals(2L, stats.getContextStats().get(contextA.name).getCompilations());
+        assertEquals(1L, stats.getContextStats().get(contextA.name).getCacheEvictions());
+        assertEquals(1L, stats.getContextStats().get(contextA.name).getCompilationLimitTriggered());
+
+        assertEquals(3L, stats.getContextStats().get(contextB.name).getCompilations());
+        assertEquals(1L, stats.getContextStats().get(contextB.name).getCacheEvictions());
+        assertEquals(2L, stats.getContextStats().get(contextB.name).getCompilationLimitTriggered());
+        assertNull(scriptService.cacheStats().getGeneralStats());
+
+        // Summed up
+        assertEquals(5L, scriptService.stats().getCompilations());
+        assertEquals(2L, scriptService.stats().getCacheEvictions());
+        assertEquals(3L, scriptService.stats().getCompilationLimitTriggered());
     }
 
     public void testStoreScript() throws Exception {

+ 1 - 1
test/framework/src/main/java/org/elasticsearch/cluster/MockInternalClusterInfoService.java

@@ -74,7 +74,7 @@ public class MockInternalClusterInfoService extends InternalClusterInfoService {
                     .map(fsInfoPath -> diskUsageFunction.apply(discoveryNode, fsInfoPath))
                     .toArray(FsInfo.Path[]::new)), nodeStats.getTransport(),
                 nodeStats.getHttp(), nodeStats.getBreaker(), nodeStats.getScriptStats(), nodeStats.getDiscoveryStats(),
-                nodeStats.getIngestStats(), nodeStats.getAdaptiveSelectionStats());
+                nodeStats.getIngestStats(), nodeStats.getAdaptiveSelectionStats(), nodeStats.getScriptCacheStats());
         }).collect(Collectors.toList());
     }
 

+ 1 - 1
test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java

@@ -2219,7 +2219,7 @@ public final class InternalTestCluster extends TestCluster {
                 NodeService nodeService = getInstanceFromNode(NodeService.class, nodeAndClient.node);
                 CommonStatsFlags flags = new CommonStatsFlags(Flag.FieldData, Flag.QueryCache, Flag.Segments);
                 NodeStats stats = nodeService.stats(flags,
-                        false, false, false, false, false, false, false, false, false, false, false, false);
+                        false, false, false, false, false, false, false, false, false, false, false, false, false);
                 assertThat("Fielddata size must be 0 on node: " + stats.getNode(),
                         stats.getIndices().getFieldData().getMemorySizeInBytes(), equalTo(0L));
                 assertThat("Query cache size must be 0 on node: " + stats.getNode(),

+ 1 - 1
x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MachineLearningInfoTransportActionTests.java

@@ -571,7 +571,7 @@ public class MachineLearningInfoTransportActionTests extends ESTestCase {
             IntStream.range(0, pipelineNames.size()).boxed().collect(Collectors.toMap(pipelineNames::get, processorStats::get)));
         return new NodeStats(mock(DiscoveryNode.class),
             Instant.now().toEpochMilli(), null, null, null, null, null, null, null, null,
-            null, null, null, ingestStats, null);
+            null, null, null, ingestStats, null, null);
 
     }
 

+ 1 - 1
x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportGetTrainedModelsStatsActionTests.java

@@ -293,7 +293,7 @@ public class TransportGetTrainedModelsStatsActionTests extends ESTestCase {
             IntStream.range(0, pipelineids.size()).boxed().collect(Collectors.toMap(pipelineids::get, processorStats::get)));
         return new NodeStats(mock(DiscoveryNode.class),
             Instant.now().toEpochMilli(), null, null, null, null, null, null, null, null,
-            null, null, null, ingestStats, null);
+            null, null, null, ingestStats, null, null);
 
     }
 

+ 1 - 1
x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/node/NodeStatsMonitoringDocTests.java

@@ -369,6 +369,6 @@ public class NodeStatsMonitoringDocTests extends BaseFilteredMonitoringDocTestCa
                                                                 emptySet(),
                                                                 Version.CURRENT);
 
-        return new NodeStats(discoveryNode, no, indices, os, process, jvm, threadPool, fs, null, null, null, null, null, null, null);
+        return new NodeStats(discoveryNode, no, indices, os, process, jvm, threadPool, fs, null, null, null, null, null, null, null, null);
     }
 }