From 118de21b662ab08d5ef49ce68bca0053dcb9ad1c Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 2 Mar 2017 17:40:28 -0500 Subject: [PATCH 01/37] uhg --- .../resources/checkstyle_suppressions.xml | 4 - .../org/elasticsearch/index/IndexModule.java | 31 +- .../org/elasticsearch/index/IndexService.java | 10 +- .../index/query/QueryRewriteContext.java | 12 +- .../index/query/QueryShardContext.java | 7 +- .../elasticsearch/indices/IndicesService.java | 11 +- .../java/org/elasticsearch/node/Node.java | 4 +- .../org/elasticsearch/node/NodeService.java | 2 +- .../elasticsearch/plugins/ScriptPlugin.java | 10 + .../elasticsearch/script/CachingCompiler.java | 402 ++++++++++++ .../script/ScriptContextRegistry.java | 8 +- .../elasticsearch/script/ScriptMetaData.java | 2 +- .../org/elasticsearch/script/ScriptModes.java | 3 +- .../elasticsearch/script/ScriptModule.java | 82 ++- .../elasticsearch/script/ScriptPermits.java | 100 +++ .../elasticsearch/script/ScriptService.java | 615 +++++------------- .../elasticsearch/script/ScriptSettings.java | 3 +- .../elasticsearch/script/TemplateService.java | 141 ++++ .../action/update/UpdateRequestTests.java | 5 +- .../elasticsearch/index/IndexModuleTests.java | 9 +- .../index/mapper/DateFieldTypeTests.java | 13 +- .../index/mapper/RangeFieldTypeTests.java | 2 +- .../index/query/QueryShardContextTests.java | 2 +- .../index/query/RangeQueryRewriteTests.java | 6 +- .../index/query/SimpleQueryParserTests.java | 4 +- .../elasticsearch/script/FileScriptTests.java | 3 +- .../script/NativeScriptTests.java | 19 +- .../script/ScriptContextTests.java | 3 +- .../script/ScriptPermitsTests.java | 54 ++ .../script/ScriptServiceTests.java | 86 ++- .../bucket/histogram/ExtendedBoundsTests.java | 2 +- .../scripted/InternalScriptedMetricTests.java | 3 +- .../ScriptedMetricAggregatorTests.java | 7 +- .../highlight/HighlightBuilderTests.java | 2 +- .../ShardSearchTransportRequestTests.java | 2 +- .../rescore/QueryRescoreBuilderTests.java | 2 +- .../search/sort/AbstractSortTestCase.java | 9 +- .../script/mustache/MustachePlugin.java | 5 +- .../mustache/MustacheScriptEngineService.java | 4 +- .../mustache/MustacheTemplateService.java | 41 ++ .../mustache/PutSearchTemplateAction.java | 42 ++ .../mustache/PutSearchTemplateRequest.java | 108 +++ .../PutSearchTemplateRequestBuilder.java | 31 + .../mustache/PutSearchTemplateResponse.java | 47 ++ .../TransportSearchTemplateAction.java | 16 +- .../mustache/TemplateQueryBuilderTests.java | 1 - .../test/lang_mustache/10_basic.yaml | 2 +- .../ingest/AbstractScriptTestCase.java | 3 +- .../test/AbstractQueryTestCase.java | 6 +- .../org/elasticsearch/test/ESTestCase.java | 2 +- .../search/MockSearchServiceTests.java | 2 +- 51 files changed, 1360 insertions(+), 630 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/script/CachingCompiler.java create mode 100644 core/src/main/java/org/elasticsearch/script/ScriptPermits.java create mode 100644 core/src/main/java/org/elasticsearch/script/TemplateService.java create mode 100644 core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheTemplateService.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateAction.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequest.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequestBuilder.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateResponse.java diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index fd2d0f1ba5501..d2a393b1d8630 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -427,10 +427,6 @@ - - - - diff --git a/core/src/main/java/org/elasticsearch/index/IndexModule.java b/core/src/main/java/org/elasticsearch/index/IndexModule.java index dc7021e81fcaf..7b351bba55ca2 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexModule.java +++ b/core/src/main/java/org/elasticsearch/index/IndexModule.java @@ -21,7 +21,6 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.TriFunction; import org.elasticsearch.common.settings.Setting; @@ -50,6 +49,7 @@ import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.TemplateService; import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; @@ -320,20 +320,19 @@ public interface IndexSearcherWrapperFactory { } public IndexService newIndexService( - NodeEnvironment environment, - NamedXContentRegistry xContentRegistry, - IndexService.ShardStoreDeleter shardStoreDeleter, - CircuitBreakerService circuitBreakerService, - BigArrays bigArrays, - ThreadPool threadPool, - ScriptService scriptService, - ClusterService clusterService, - Client client, - IndicesQueryCache indicesQueryCache, - MapperRegistry mapperRegistry, - Consumer globalCheckpointSyncer, - IndicesFieldDataCache indicesFieldDataCache) - throws IOException { + NodeEnvironment environment, + NamedXContentRegistry xContentRegistry, + IndexService.ShardStoreDeleter shardStoreDeleter, + CircuitBreakerService circuitBreakerService, + BigArrays bigArrays, + ThreadPool threadPool, + ScriptService scriptService, + TemplateService templateService, + Client client, + IndicesQueryCache indicesQueryCache, + MapperRegistry mapperRegistry, + Consumer globalCheckpointSyncer, + IndicesFieldDataCache indicesFieldDataCache) throws IOException { final IndexEventListener eventListener = freeze(); IndexSearcherWrapperFactory searcherWrapperFactory = indexSearcherWrapper.get() == null ? (shard) -> null : indexSearcherWrapper.get(); @@ -365,7 +364,7 @@ public IndexService newIndexService( } return new IndexService(indexSettings, environment, xContentRegistry, new SimilarityService(indexSettings, similarities), shardStoreDeleter, analysisRegistry, engineFactory.get(), circuitBreakerService, bigArrays, threadPool, scriptService, - clusterService, client, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry, + templateService, client, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry, indicesFieldDataCache, globalCheckpointSyncer, searchOperationListeners, indexOperationListeners); } diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index ee35993c01e79..c64311af5a054 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -28,7 +28,6 @@ import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; @@ -68,6 +67,7 @@ import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.TemplateService; import org.elasticsearch.threadpool.ThreadPool; import java.io.Closeable; @@ -118,7 +118,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final BigArrays bigArrays; private final AsyncGlobalCheckpointTask globalCheckpointTask; private final ScriptService scriptService; - private final ClusterService clusterService; + private final TemplateService templateService; private final Client client; public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv, @@ -131,7 +131,7 @@ public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv, BigArrays bigArrays, ThreadPool threadPool, ScriptService scriptService, - ClusterService clusterService, + TemplateService templateService, Client client, QueryCache queryCache, IndexStore indexStore, @@ -158,7 +158,7 @@ public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv, this.bigArrays = bigArrays; this.threadPool = threadPool; this.scriptService = scriptService; - this.clusterService = clusterService; + this.templateService = templateService; this.client = client; this.eventListener = eventListener; this.nodeEnv = nodeEnv; @@ -473,7 +473,7 @@ public IndexSettings getIndexSettings() { public QueryShardContext newQueryShardContext(int shardId, IndexReader indexReader, LongSupplier nowInMillis) { return new QueryShardContext( shardId, indexSettings, indexCache.bitsetFilterCache(), indexFieldData, mapperService(), - similarityService(), scriptService, xContentRegistry, + similarityService(), scriptService, templateService, xContentRegistry, client, indexReader, nowInMillis); } diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java index 80726496a739c..4463cae006700 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java @@ -29,6 +29,7 @@ import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.TemplateService; import java.util.function.LongSupplier; @@ -38,6 +39,7 @@ public class QueryRewriteContext { protected final MapperService mapperService; protected final ScriptService scriptService; + private final TemplateService templateService; protected final IndexSettings indexSettings; private final NamedXContentRegistry xContentRegistry; protected final Client client; @@ -45,10 +47,11 @@ public class QueryRewriteContext { protected final LongSupplier nowInMillis; public QueryRewriteContext(IndexSettings indexSettings, MapperService mapperService, ScriptService scriptService, - NamedXContentRegistry xContentRegistry, Client client, IndexReader reader, + TemplateService templateService, NamedXContentRegistry xContentRegistry, Client client, IndexReader reader, LongSupplier nowInMillis) { this.mapperService = mapperService; this.scriptService = scriptService; + this.templateService = templateService; this.indexSettings = indexSettings; this.xContentRegistry = xContentRegistry; this.client = client; @@ -104,7 +107,12 @@ public long nowInMillis() { } public BytesReference getTemplateBytes(Script template) { - ExecutableScript executable = scriptService.executable(template, ScriptContext.Standard.SEARCH); + ExecutableScript executable = templateService.executable(template.getIdOrCode(), template.getType(), ScriptContext.Standard.SEARCH, + template.getParams()); return (BytesReference) executable.run(); } + + public TemplateService getTemplateService() { + return templateService; + } } diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index 2b5e69947f373..76a4d9a549d1f 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -56,6 +56,7 @@ import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.SearchScript; +import org.elasticsearch.script.TemplateService; import org.elasticsearch.search.lookup.SearchLookup; import java.io.IOException; @@ -100,9 +101,9 @@ public String[] getTypes() { public QueryShardContext(int shardId, IndexSettings indexSettings, BitsetFilterCache bitsetFilterCache, IndexFieldDataService indexFieldDataService, MapperService mapperService, SimilarityService similarityService, - ScriptService scriptService, NamedXContentRegistry xContentRegistry, + ScriptService scriptService, TemplateService templateService, NamedXContentRegistry xContentRegistry, Client client, IndexReader reader, LongSupplier nowInMillis) { - super(indexSettings, mapperService, scriptService, xContentRegistry, client, reader, nowInMillis); + super(indexSettings, mapperService, scriptService, templateService, xContentRegistry, client, reader, nowInMillis); this.shardId = shardId; this.indexSettings = indexSettings; this.similarityService = similarityService; @@ -116,7 +117,7 @@ public QueryShardContext(int shardId, IndexSettings indexSettings, BitsetFilterC public QueryShardContext(QueryShardContext source) { this(source.shardId, source.indexSettings, source.bitsetFilterCache, source.indexFieldDataService, source.mapperService, - source.similarityService, source.scriptService, source.getXContentRegistry(), source.client, + source.similarityService, source.scriptService, source.getTemplateService(), source.getXContentRegistry(), source.client, source.reader, source.nowInMillis); this.types = source.getTypes(); } diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index a4e4c83bc0079..10fafa898c28d 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -41,7 +41,6 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.breaker.CircuitBreaker; @@ -110,6 +109,7 @@ import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.TemplateService; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchRequest; @@ -121,7 +121,6 @@ import java.io.IOException; import java.nio.file.Files; import java.util.ArrayList; -import java.util.EnumSet; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -162,7 +161,7 @@ public class IndicesService extends AbstractLifecycleComponent private final CircuitBreakerService circuitBreakerService; private final BigArrays bigArrays; private final ScriptService scriptService; - private final ClusterService clusterService; + private final TemplateService templateService; private final Client client; private volatile Map indices = emptyMap(); private final Map> pendingDeletes = new HashMap<>(); @@ -187,7 +186,7 @@ public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvi IndexNameExpressionResolver indexNameExpressionResolver, MapperRegistry mapperRegistry, NamedWriteableRegistry namedWriteableRegistry, ThreadPool threadPool, IndexScopedSettings indexScopedSettings, CircuitBreakerService circuitBreakerService, - BigArrays bigArrays, ScriptService scriptService, ClusterService clusterService, Client client, + BigArrays bigArrays, ScriptService scriptService, TemplateService templateService, Client client, MetaStateService metaStateService) { super(settings); this.threadPool = threadPool; @@ -208,7 +207,7 @@ public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvi this.circuitBreakerService = circuitBreakerService; this.bigArrays = bigArrays; this.scriptService = scriptService; - this.clusterService = clusterService; + this.templateService = templateService; this.client = client; this.indicesFieldDataCache = new IndicesFieldDataCache(settings, new IndexFieldDataCache.Listener() { @Override @@ -447,7 +446,7 @@ private synchronized IndexService createIndexService(final String reason, bigArrays, threadPool, scriptService, - clusterService, + templateService, client, indicesQueryCache, mapperRegistry, diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 00e00b745a09b..3cfe4157c2395 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -117,6 +117,7 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.script.ScriptModule; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.TemplateService; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.fetch.FetchPhase; @@ -389,7 +390,7 @@ protected Node(final Environment environment, Collection settingsModule.getClusterSettings(), analysisModule.getAnalysisRegistry(), clusterModule.getIndexNameExpressionResolver(), indicesModule.getMapperRegistry(), namedWriteableRegistry, threadPool, settingsModule.getIndexScopedSettings(), circuitBreakerService, bigArrays, scriptModule.getScriptService(), - clusterService, client, metaStateService); + scriptModule.getTemplateService(), client, metaStateService); Collection pluginComponents = pluginsService.filterPlugins(Plugin.class).stream() .flatMap(p -> p.createComponents(client, clusterService, threadPool, resourceWatcherService, @@ -439,6 +440,7 @@ protected Node(final Environment environment, Collection b.bind(CircuitBreakerService.class).toInstance(circuitBreakerService); b.bind(BigArrays.class).toInstance(bigArrays); b.bind(ScriptService.class).toInstance(scriptModule.getScriptService()); + b.bind(TemplateService.class).toInstance(scriptModule.getTemplateService()); b.bind(AnalysisRegistry.class).toInstance(analysisModule.getAnalysisRegistry()); b.bind(IngestService.class).toInstance(ingestService); b.bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry); diff --git a/core/src/main/java/org/elasticsearch/node/NodeService.java b/core/src/main/java/org/elasticsearch/node/NodeService.java index cb245487152e3..3053d8707bbe8 100644 --- a/core/src/main/java/org/elasticsearch/node/NodeService.java +++ b/core/src/main/java/org/elasticsearch/node/NodeService.java @@ -53,7 +53,7 @@ public class NodeService extends AbstractComponent implements Closeable { private final CircuitBreakerService circuitBreakerService; private final IngestService ingestService; private final SettingsFilter settingsFilter; - private ScriptService scriptService; + private final ScriptService scriptService; private final HttpServerTransport httpServerTransport; diff --git a/core/src/main/java/org/elasticsearch/plugins/ScriptPlugin.java b/core/src/main/java/org/elasticsearch/plugins/ScriptPlugin.java index c1e2a43c95365..078559f919881 100644 --- a/core/src/main/java/org/elasticsearch/plugins/ScriptPlugin.java +++ b/core/src/main/java/org/elasticsearch/plugins/ScriptPlugin.java @@ -22,6 +22,7 @@ import org.elasticsearch.script.NativeScriptFactory; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptEngineService; +import org.elasticsearch.script.TemplateService; import java.util.Collections; import java.util.List; @@ -38,6 +39,15 @@ default ScriptEngineService getScriptEngineService(Settings settings) { return null; } + /** + * Returns a {@link TemplateService.Backend} if this plugin implements a template backend or null if it doesn't. Note that Elasticsearch + * will refuse to start if there is more than one template backend and it is bundled with Mustache. To replace that backend you'd have + * to remove the Mustache module which is super untested. + */ + default TemplateService.Backend getTemplateBackend() { + return null; + } + /** * Returns a list of {@link NativeScriptFactory} instances. */ diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java new file mode 100644 index 0000000000000..7f0c7f099c926 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -0,0 +1,402 @@ +/* + * 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.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.logging.log4j.util.Supplier; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ResourceNotFoundException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.storedscripts.DeleteStoredScriptRequest; +import org.elasticsearch.action.admin.cluster.storedscripts.DeleteStoredScriptResponse; +import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptRequest; +import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptRequest; +import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptResponse; +import org.elasticsearch.cluster.AckedClusterStateUpdateTask; +import org.elasticsearch.cluster.ClusterChangedEvent; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.cache.Cache; +import org.elasticsearch.common.cache.CacheBuilder; +import org.elasticsearch.common.cache.RemovalListener; +import org.elasticsearch.common.cache.RemovalNotification; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.logging.ESLoggerFactory; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.env.Environment; +import org.elasticsearch.watcher.FileChangesListener; +import org.elasticsearch.watcher.FileWatcher; +import org.elasticsearch.watcher.ResourceWatcherService; + +import java.io.IOException; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Objects; +import java.util.concurrent.ConcurrentMap; + +/** + * Manages caching, resource watching, permissions checking, and compilation of scripts (or templates). + */ +public abstract class CachingCompiler implements ClusterStateListener { + private static final Logger logger = ESLoggerFactory.getLogger(CachingCompiler.class); + + /** + * Compiled file scripts (or templates). Modified by the file watching process. + */ + private final ConcurrentMap fileScripts = ConcurrentCollections.newConcurrentMap(); + + /** + * Cache of compiled dynamic scripts (or templates). + */ + private final Cache cache; + + private final Path scriptsDirectory; + + private final int maxScriptSizeInBytes; + + private final ScriptMetrics scriptMetrics; + + private volatile ClusterState clusterState; + + public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environment env, + ResourceWatcherService resourceWatcherService, ScriptMetrics scriptMetrics) throws IOException { + int cacheMaxSize = ScriptService.SCRIPT_CACHE_SIZE_SETTING.get(settings); + + CacheBuilder cacheBuilder = CacheBuilder.builder(); + if (cacheMaxSize >= 0) { + cacheBuilder.setMaximumWeight(cacheMaxSize); + } + + TimeValue cacheExpire = ScriptService.SCRIPT_CACHE_EXPIRE_SETTING.get(settings); + if (cacheExpire.getNanos() != 0) { + cacheBuilder.setExpireAfterAccess(cacheExpire); + } + + logger.debug("using script cache with max_size [{}], expire [{}]", cacheMaxSize, cacheExpire); + this.cache = cacheBuilder.removalListener(new ScriptCacheRemovalListener()).build(); + + // add file watcher for file scripts + scriptsDirectory = env.scriptsFile(); + if (logger.isTraceEnabled()) { + logger.trace("Using scripts directory [{}] ", scriptsDirectory); + } + FileWatcher fileWatcher = new FileWatcher(scriptsDirectory); + fileWatcher.addListener(new ScriptsDirectoryChangesListener()); + if (ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.get(settings)) { + // automatic reload is enabled - register scripts + resourceWatcherService.add(fileWatcher); + } else { + // automatic reload is disabled just load scripts once + fileWatcher.init(); + } + + maxScriptSizeInBytes = ScriptService.SCRIPT_MAX_SIZE_IN_BYTES.get(settings); + this.scriptMetrics = scriptMetrics; + } + + /** + * Build the cache key for a file name and its extension. Return null to indicate that the file type is not supported. + */ // NOCOMMIT rename to cacheKeyForFile + protected abstract CacheKeyT cacheKeyForFile(String baseName, String extension); + protected abstract CacheKeyT cacheKeyFromClusterState(StoredScriptSource scriptMetadata); + protected abstract StoredScriptSource lookupStoredScript(ClusterState clusterState, CacheKeyT cacheKey); + protected abstract void checkPutSupported(StoredScriptSource source); + /** + * Are any script contexts enabled for the given {@code cacheKey} and {@code scriptType}? Used to reject compilation if all script + * contexts are disabled and produce a nice error message earlier rather than later. + */ // NOCOMMIT make sure we have tests for cases where we use this (known cases are files and cluster state) + protected abstract boolean areAnyScriptContextsEnabled(CacheKeyT cacheKey, ScriptType scriptType); + /** + * Check if a script can be executed. + */ + protected abstract void checkCanExecuteScript(CacheKeyT cacheKey, ScriptType scriptType, ScriptContext scriptContext); + /** + * Check if too many scripts (or templates) have been compiled recently. + */ + protected abstract void checkCompilationLimit(); + // NOCOMMIT document + protected abstract CompiledScript compile(ScriptType scriptType, CacheKeyT cacheKey); + protected abstract CompiledScript compileFileScript(CacheKeyT cacheKey, String body, Path file); + + public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, ScriptContext scriptContext) { + Objects.requireNonNull(cacheKey); + + // First resolve stored scripts so so we have accurate parameters for checkCanExecuteScript + if (scriptType == ScriptType.STORED) { + cacheKey = getScriptFromClusterState(cacheKey); + } + + // Validate that we can execute the script + checkCanExecuteScript(cacheKey, scriptType, scriptContext); + + // Lookup file scripts from the map we maintain by watching the directory + if (scriptType == ScriptType.FILE) { + CompiledScript compiled = fileScripts.get(cacheKey); + if (compiled == null) { + throw new IllegalArgumentException("unable to find file script " + cacheKey); + } + return compiled; + } + + // Other scripts are compiled lazily when needed so check the cache first + CompiledScript compiledScript = cache.get(cacheKey); + if (compiledScript != null) { + return compiledScript; + } + + // Synchronize so we don't compile scripts many times during multiple shards all compiling a script + synchronized (this) { + // Double check in case it was compiled while we were waiting for the monitor + compiledScript = cache.get(cacheKey); + if (compiledScript != null) { + return compiledScript; + } + + try { + if (logger.isTraceEnabled()) { + logger.trace("compiling [{}]", cacheKey); + } + // Check whether too many compilations have happened + checkCompilationLimit(); + compiledScript = compile(scriptType, cacheKey); + } catch (ScriptException good) { + // TODO: remove this try-catch completely, when all script engines have good exceptions! + throw good; // its already good + } catch (Exception exception) { + throw new GeneralScriptException("Failed to compile " + cacheKey, exception); + } + scriptMetrics.onCompilation(); + cache.put(cacheKey, compiledScript); + return compiledScript; + } + } + + private CacheKeyT getScriptFromClusterState(CacheKeyT cacheKey) { + StoredScriptSource source = lookupStoredScript(clusterState, cacheKey); + + if (source == null) { + throw new ResourceNotFoundException("unable to find script [" + cacheKey + "] in cluster state"); + } + + return cacheKeyFromClusterState(source); + } + + public final void putStoredScript(ClusterService clusterService, PutStoredScriptRequest request, + ActionListener listener) { + if (request.content().length() > maxScriptSizeInBytes) { + throw new IllegalArgumentException("exceeded max allowed stored script size in bytes [" + maxScriptSizeInBytes + + "] with size [" + request.content().length() + "] for script [" + request.id() + "]"); + } + + StoredScriptSource source = StoredScriptSource.parse(request.lang(), request.content(), request.xContentType()); + checkPutSupported(source); + CacheKeyT cacheKey = cacheKeyFromClusterState(source); + + try { + if (areAnyScriptContextsEnabled(cacheKey, ScriptType.STORED)) { + Object compiled = compile(ScriptType.STORED, cacheKey); + + if (compiled == null) { + throw new IllegalArgumentException("failed to parse/compile stored script [" + request.id() + "]" + + (source.getCode() == null ? "" : " using code [" + source.getCode() + "]")); + } + } else { + throw new IllegalArgumentException( + "cannot put stored script [" + request.id() + "], stored scripts cannot be run under any context"); + } + } catch (ScriptException good) { + throw good; + } catch (Exception exception) { + throw new IllegalArgumentException("failed to parse/compile stored script [" + request.id() + "]", exception); + } + + clusterService.submitStateUpdateTask("put-script-" + request.id(), + new AckedClusterStateUpdateTask(request, listener) { + + @Override + protected PutStoredScriptResponse newResponse(boolean acknowledged) { + return new PutStoredScriptResponse(acknowledged); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); + smd = ScriptMetaData.putStoredScript(smd, request.id(), source); + MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); + + return ClusterState.builder(currentState).metaData(mdb).build(); + } + }); + } + + public final void deleteStoredScript(ClusterService clusterService, DeleteStoredScriptRequest request, + ActionListener listener) { + clusterService.submitStateUpdateTask("delete-script-" + request.id(), + new AckedClusterStateUpdateTask(request, listener) { + @Override + protected DeleteStoredScriptResponse newResponse(boolean acknowledged) { + return new DeleteStoredScriptResponse(acknowledged); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); + smd = ScriptMetaData.deleteStoredScript(smd, request.id(), request.lang()); + MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); + + return ClusterState.builder(currentState).metaData(mdb).build(); + } + }); + } + + public final StoredScriptSource getStoredScript(ClusterState state, GetStoredScriptRequest request) { + ScriptMetaData scriptMetadata = state.metaData().custom(ScriptMetaData.TYPE); + + if (scriptMetadata != null) { + return scriptMetadata.getStoredScript(request.id(), request.lang()); + } else { + return null; + } + } + + @Override + public final void clusterChanged(ClusterChangedEvent event) { + clusterState = event.state(); + } + + /** + * Listener to manage metrics for the script cache. + */ + private class ScriptCacheRemovalListener implements RemovalListener { + @Override + public void onRemoval(RemovalNotification notification) { + if (logger.isDebugEnabled()) { + logger.debug("removed {} from cache, reason: {}", notification.getValue(), notification.getRemovalReason()); + } + scriptMetrics.onCacheEviction(); + } + } + + private class ScriptsDirectoryChangesListener implements FileChangesListener { + private Tuple getScriptNameExt(Path file) { + Path scriptPath = scriptsDirectory.relativize(file); + int extIndex = scriptPath.toString().lastIndexOf('.'); + if (extIndex <= 0) { + return null; + } + + String ext = scriptPath.toString().substring(extIndex + 1); + if (ext.isEmpty()) { + return null; + } + + String scriptName = scriptPath.toString().substring(0, extIndex).replace(scriptPath.getFileSystem().getSeparator(), "_"); + return new Tuple<>(scriptName, ext); + } + + @Override + public void onFileInit(Path file) { + Tuple scriptNameExt = getScriptNameExt(file); + if (scriptNameExt == null) { + logger.debug("Skipped script with invalid extension : [{}]", file); + return; + } + if (logger.isTraceEnabled()) { + logger.trace("Loading script file : [{}]", file); + } + + CacheKeyT cacheKey = cacheKeyForFile(scriptNameExt.v1(), scriptNameExt.v2()); + if (cacheKey == null) { + logger.warn("No script engine found for [{}]", scriptNameExt.v2()); + return; + } + try { + /* we don't know yet what the script will be used for, but if all of the operations for this lang with file scripts are + * disabled, it makes no sense to even compile it and cache it. */ + if (areAnyScriptContextsEnabled(cacheKey, ScriptType.FILE)) { + logger.info("compiling script file [{}]", file.toAbsolutePath()); + try (InputStreamReader reader = new InputStreamReader(Files.newInputStream(file), StandardCharsets.UTF_8)) { + String body = Streams.copyToString(reader); + fileScripts.put(cacheKey, compileFileScript(cacheKey, body, file)); + scriptMetrics.onCompilation(); + } + } else { + logger.warn("skipping compile of script file [{}] as all scripted operations are disabled for file scripts", + + file.toAbsolutePath()); + } + } catch (ScriptException e) { + /* Attempt to extract a concise error message using the xcontent generation mechanisms and log that. */ + try (XContentBuilder builder = JsonXContent.contentBuilder()) { + builder.prettyPrint(); + builder.startObject(); + ElasticsearchException.generateThrowableXContent(builder, ToXContent.EMPTY_PARAMS, e); + builder.endObject(); + logger.warn("failed to load/compile script [{}]: {}", scriptNameExt.v1(), builder.string()); + } catch (IOException ioe) { + ioe.addSuppressed(e); + logger.warn((Supplier) () -> new ParameterizedMessage( + "failed to log an appropriate warning after failing to load/compile script [{}]", scriptNameExt.v1()), ioe); + } + /* Log at the whole exception at the debug level as well just in case the stack trace is important. That way you can + * turn on the stack trace if you need it. */ + logger.debug((Supplier) () -> new ParameterizedMessage("failed to load/compile script [{}]. full exception:", + scriptNameExt.v1()), e); + } catch (Exception e) { + logger.warn((Supplier) () -> new ParameterizedMessage("failed to load/compile script [{}]", scriptNameExt.v1()), e); + } + } + + @Override + public void onFileCreated(Path file) { + onFileInit(file); + } + + @Override + public void onFileDeleted(Path file) { + Tuple scriptNameExt = getScriptNameExt(file); + if (scriptNameExt == null) { + return; + } + CacheKeyT cacheKey = cacheKeyForFile(scriptNameExt.v1(), scriptNameExt.v2()); + if (cacheKey == null) { + return; + } + logger.info("removing script file [{}]", file.toAbsolutePath()); + fileScripts.remove(cacheKey); + } + + @Override + public void onFileChanged(Path file) { + onFileInit(file); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/script/ScriptContextRegistry.java b/core/src/main/java/org/elasticsearch/script/ScriptContextRegistry.java index 2b7feeb8d7fdf..2d1e0c1e6736a 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptContextRegistry.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptContextRegistry.java @@ -57,7 +57,7 @@ public ScriptContextRegistry(Collection customScriptContex /** * @return a list that contains all the supported {@link ScriptContext}s, both standard ones and registered via plugins */ - Collection scriptContexts() { + public Collection scriptContexts() { return scriptContexts.values(); } @@ -71,10 +71,12 @@ boolean isSupportedContext(ScriptContext scriptContext) { //script contexts can be used in fine-grained settings, we need to be careful with what we allow here private void validateScriptContext(ScriptContext.Plugin scriptContext) { if (RESERVED_SCRIPT_CONTEXTS.contains(scriptContext.getPluginName())) { - throw new IllegalArgumentException("[" + scriptContext.getPluginName() + "] is a reserved name, it cannot be registered as a custom script context"); + throw new IllegalArgumentException("[" + scriptContext.getPluginName() + + "] is a reserved name, it cannot be registered as a custom script context"); } if (RESERVED_SCRIPT_CONTEXTS.contains(scriptContext.getOperation())) { - throw new IllegalArgumentException("[" + scriptContext.getOperation() + "] is a reserved name, it cannot be registered as a custom script context"); + throw new IllegalArgumentException("[" + scriptContext.getOperation() + + "] is a reserved name, it cannot be registered as a custom script context"); } } diff --git a/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java b/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java index 87afc21a8c020..785c8f30782bd 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java @@ -425,7 +425,7 @@ public EnumSet context() { * Otherwise, returns a stored script from the deprecated namespace. Either * way an id is required. */ - StoredScriptSource getStoredScript(String id, String lang) { + public StoredScriptSource getStoredScript(String id, String lang) { if (lang == null) { return scripts.get(id); } else { diff --git a/core/src/main/java/org/elasticsearch/script/ScriptModes.java b/core/src/main/java/org/elasticsearch/script/ScriptModes.java index 15393948d6654..439a09848d4cb 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptModes.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptModes.java @@ -62,7 +62,8 @@ public boolean getScriptEnabled(String lang, ScriptType scriptType, ScriptContex } Boolean scriptMode = scriptEnabled.get(getKey(lang, scriptType, scriptContext)); if (scriptMode == null) { - throw new IllegalArgumentException("script mode not found for lang [" + lang + "], script_type [" + scriptType + "], operation [" + scriptContext.getKey() + "]"); + throw new IllegalArgumentException("script mode not found for lang [" + lang + "], script_type [" + + scriptType + "], operation [" + scriptContext.getKey() + "]"); } return scriptMode; } diff --git a/core/src/main/java/org/elasticsearch/script/ScriptModule.java b/core/src/main/java/org/elasticsearch/script/ScriptModule.java index 30f84bc6960bc..59d13a9ba04a0 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptModule.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptModule.java @@ -19,16 +19,16 @@ package org.elasticsearch.script; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.plugins.ScriptPlugin; +import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.watcher.ResourceWatcherService; import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -39,12 +39,10 @@ * Manages building {@link ScriptService} and {@link ScriptSettings} from a list of plugins. */ public class ScriptModule { - private final ScriptSettings scriptSettings; - private final ScriptService scriptService; - /** * Build from {@linkplain ScriptPlugin}s. Convenient for normal use but not great for tests. See - * {@link ScriptModule#ScriptModule(Settings, Environment, ResourceWatcherService, List, List)} for easier use in tests. + * {@link ScriptModule#ScriptModule(Settings, Environment, ResourceWatcherService, List, List, TemplateService.Backend)} + * for easier use in tests. */ public static ScriptModule create(Settings settings, Environment environment, ResourceWatcherService resourceWatcherService, List scriptPlugins) { @@ -56,21 +54,49 @@ public static ScriptModule create(Settings settings, Environment environment, scriptEngineServices.add(nativeScriptEngineService); List plugins = scriptPlugins.stream().map(x -> x.getCustomScriptContexts()).filter(Objects::nonNull) .collect(Collectors.toList()); - return new ScriptModule(settings, environment, resourceWatcherService, scriptEngineServices, plugins); + List templateBackends = scriptPlugins.stream().map(x -> x.getTemplateBackend()) + .filter(Objects::nonNull).collect(Collectors.toList()); + TemplateService.Backend templateBackend; + switch (templateBackends.size()) { + case 0: + templateBackend = null; + break; + case 1: + templateBackend = templateBackends.get(0); + break; + default: + throw new IllegalArgumentException("Elasticsearch only supports a single template backend but was started with [" + + templateBackends + "]"); + } + return new ScriptModule(settings, environment, resourceWatcherService, scriptEngineServices, plugins, templateBackend); } + private final ScriptSettings scriptSettings; + private final ScriptService scriptService; + private final TemplateService templateService; + /** * Build {@linkplain ScriptEngineService} and {@linkplain ScriptContext.Plugin}. */ public ScriptModule(Settings settings, Environment environment, ResourceWatcherService resourceWatcherService, List scriptEngineServices, - List customScriptContexts) { + List customScriptContexts, @Nullable TemplateService.Backend templateBackend) { ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(customScriptContexts); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(scriptEngineServices); + ScriptMetrics scriptMetrics = new ScriptMetrics(); scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); try { scriptService = new ScriptService(settings, environment, resourceWatcherService, scriptEngineRegistry, scriptContextRegistry, - scriptSettings); + scriptSettings, scriptMetrics); + } catch (IOException e) { + throw new RuntimeException("Couldn't setup ScriptService", e); + } + if (templateBackend == null) { + templateBackend = new TemplatesUnsupportedBackend(); + } + try { + templateService = new TemplateService(settings, environment, resourceWatcherService, templateBackend, + scriptContextRegistry, scriptSettings, scriptMetrics); } catch (IOException e) { throw new RuntimeException("Couldn't setup ScriptService", e); } @@ -90,10 +116,48 @@ public ScriptService getScriptService() { return scriptService; } + /** + * The service responsible for managing templates. + */ + public TemplateService getTemplateService() { + return templateService; + } + /** * Allow the script service to register any settings update handlers on the cluster settings */ public void registerClusterSettingsListeners(ClusterSettings clusterSettings) { scriptService.registerClusterSettingsListeners(clusterSettings); } + + private static class TemplatesUnsupportedBackend implements TemplateService.Backend { + @Override + public String getType() { + throw new UnsupportedOperationException("templates are unsupported"); + } + + @Override + public String getExtension() { + throw new UnsupportedOperationException("templates are unsupported"); + } + + @Override + public Object compile(String scriptName, String scriptSource, Map params) { + throw new UnsupportedOperationException("templates are unsupported"); + } + + @Override + public ExecutableScript executable(CompiledScript compiledScript, Map vars) { + throw new UnsupportedOperationException("templates are unsupported"); + } + + @Override + public SearchScript search(CompiledScript compiledScript, SearchLookup lookup, Map vars) { + throw new UnsupportedOperationException("templates are unsupported"); + } + + @Override + public void close() throws IOException { + } + } } diff --git a/core/src/main/java/org/elasticsearch/script/ScriptPermits.java b/core/src/main/java/org/elasticsearch/script/ScriptPermits.java new file mode 100644 index 0000000000000..a6c1f90b5c5ec --- /dev/null +++ b/core/src/main/java/org/elasticsearch/script/ScriptPermits.java @@ -0,0 +1,100 @@ +/* + * 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.breaker.CircuitBreakingException; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; + +/** + * Encapsulates logic for checking if a script is allowed to be compiled. Has two kinds of checks: + *
    + *
  1. Is this script allowed to be compiled in this context? + *
  2. Have there been too many inline compilations lately? + *
+ */ +public class ScriptPermits { + private final ScriptModes scriptModes; + private final ScriptContextRegistry scriptContextRegistry; + + private int totalCompilesPerMinute; + private long lastInlineCompileTime; + private double scriptsPerMinCounter; + private double compilesAllowedPerNano; + + public ScriptPermits(Settings settings, ScriptSettings scriptSettings, ScriptContextRegistry scriptContextRegistry) { + this.scriptModes = new ScriptModes(scriptSettings, settings); + this.scriptContextRegistry = scriptContextRegistry; + this.lastInlineCompileTime = System.nanoTime(); + this.setMaxCompilationsPerMinute(ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE.get(settings)); + } + + void registerClusterSettingsListeners(ClusterSettings clusterSettings) { + clusterSettings.addSettingsUpdateConsumer(ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE, this::setMaxCompilationsPerMinute); + } + + void setMaxCompilationsPerMinute(Integer newMaxPerMinute) { + this.totalCompilesPerMinute = newMaxPerMinute; + // Reset the counter to allow new compilations + this.scriptsPerMinCounter = totalCompilesPerMinute; + this.compilesAllowedPerNano = ((double) totalCompilesPerMinute) / TimeValue.timeValueMinutes(1).nanos(); + } + + /** + * Check whether there have been too many compilations within the last minute, throwing a circuit breaking exception if so. + * This is a variant of the token bucket algorithm: https://en.wikipedia.org/wiki/Token_bucket + * + * It can be thought of as a bucket with water, every time the bucket is checked, water is added proportional to the amount of time that + * elapsed since the last time it was checked. If there is enough water, some is removed and the request is allowed. If there is not + * enough water the request is denied. Just like a normal bucket, if water is added that overflows the bucket, the extra water/capacity + * is discarded - there can never be more water in the bucket than the size of the bucket. + */ + public void checkCompilationLimit() { + long now = System.nanoTime(); + long timePassed = now - lastInlineCompileTime; + lastInlineCompileTime = now; + + scriptsPerMinCounter += (timePassed) * compilesAllowedPerNano; + + // It's been over the time limit anyway, readjust the bucket to be level + if (scriptsPerMinCounter > totalCompilesPerMinute) { + scriptsPerMinCounter = totalCompilesPerMinute; + } + + // If there is enough tokens in the bucket, allow the request and decrease the tokens by 1 + if (scriptsPerMinCounter >= 1) { + scriptsPerMinCounter -= 1.0; + } else { + // Otherwise reject the request + throw new CircuitBreakingException("[script] Too many dynamic script compilations within one minute, max: [" + + totalCompilesPerMinute + "/min]; please use on-disk, indexed, or scripts with parameters instead; " + + "this limit can be changed by the [" + ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE.getKey() + "] setting"); + } + } + + public boolean canExecuteScript(String lang, ScriptType scriptType, ScriptContext scriptContext) { + assert lang != null; + if (scriptContextRegistry.isSupportedContext(scriptContext) == false) { + throw new IllegalArgumentException("script context [" + scriptContext.getKey() + "] not supported"); + } + return scriptModes.getScriptEnabled(lang, scriptType, scriptContext); + } +} diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index 692e081a7ba1a..80afb80058e02 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -19,59 +19,35 @@ package org.elasticsearch.script; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.logging.log4j.util.Supplier; import org.apache.lucene.util.IOUtils; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.storedscripts.DeleteStoredScriptRequest; import org.elasticsearch.action.admin.cluster.storedscripts.DeleteStoredScriptResponse; import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptRequest; import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptRequest; import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptResponse; -import org.elasticsearch.cluster.AckedClusterStateUpdateTask; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateListener; -import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.breaker.CircuitBreakingException; -import org.elasticsearch.common.cache.Cache; -import org.elasticsearch.common.cache.CacheBuilder; -import org.elasticsearch.common.cache.RemovalListener; -import org.elasticsearch.common.cache.RemovalNotification; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractComponent; -import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.env.Environment; import org.elasticsearch.search.lookup.SearchLookup; -import org.elasticsearch.watcher.FileChangesListener; -import org.elasticsearch.watcher.FileWatcher; import org.elasticsearch.watcher.ResourceWatcherService; import java.io.Closeable; import java.io.IOException; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; import java.nio.file.Path; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Objects; -import java.util.concurrent.ConcurrentMap; import static java.util.Collections.unmodifiableMap; @@ -94,51 +70,23 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust private final Map scriptEnginesByLang; private final Map scriptEnginesByExt; - private final ConcurrentMap staticCache = ConcurrentCollections.newConcurrentMap(); + private final ScriptMetrics scriptMetrics; + private final ScriptPermits scriptPermits; + private final CachingCompiler compiler; - private final Cache cache; - private final Path scriptsDirectory; - - private final ScriptModes scriptModes; - private final ScriptContextRegistry scriptContextRegistry; - - private final ScriptMetrics scriptMetrics = new ScriptMetrics(); - - private ClusterState clusterState; - - private int totalCompilesPerMinute; - private long lastInlineCompileTime; - private double scriptsPerMinCounter; - private double compilesAllowedPerNano; - - public ScriptService(Settings settings, Environment env, - ResourceWatcherService resourceWatcherService, ScriptEngineRegistry scriptEngineRegistry, - ScriptContextRegistry scriptContextRegistry, ScriptSettings scriptSettings) throws IOException { + public ScriptService(Settings settings, Environment env, ResourceWatcherService resourceWatcherService, + ScriptEngineRegistry scriptEngineRegistry, ScriptContextRegistry scriptContextRegistry, ScriptSettings scriptSettings, + ScriptMetrics scriptMetrics) throws IOException { super(settings); - Objects.requireNonNull(scriptEngineRegistry); - Objects.requireNonNull(scriptContextRegistry); - Objects.requireNonNull(scriptSettings); if (Strings.hasLength(settings.get(DISABLE_DYNAMIC_SCRIPTING_SETTING))) { - throw new IllegalArgumentException(DISABLE_DYNAMIC_SCRIPTING_SETTING + " is not a supported setting, replace with fine-grained script settings. \n" + - "Dynamic scripts can be enabled for all languages and all operations by replacing `script.disable_dynamic: false` with `script.inline: true` and `script.stored: true` in elasticsearch.yml"); + throw new IllegalArgumentException(DISABLE_DYNAMIC_SCRIPTING_SETTING + " is not a supported setting, replace with " + + "fine-grained script settings. Dynamic scripts can be enabled for all languages and all operations by replacing " + + "`script.disable_dynamic: false` with `script.inline: true` and `script.stored: true` in elasticsearch.yml"); } + Objects.requireNonNull(scriptEngineRegistry); this.scriptEngines = scriptEngineRegistry.getRegisteredLanguages().values(); - this.scriptContextRegistry = scriptContextRegistry; - int cacheMaxSize = SCRIPT_CACHE_SIZE_SETTING.get(settings); - - CacheBuilder cacheBuilder = CacheBuilder.builder(); - if (cacheMaxSize >= 0) { - cacheBuilder.setMaximumWeight(cacheMaxSize); - } - - TimeValue cacheExpire = SCRIPT_CACHE_EXPIRE_SETTING.get(settings); - if (cacheExpire.getNanos() != 0) { - cacheBuilder.setExpireAfterAccess(cacheExpire); - } - - logger.debug("using script cache with max_size [{}], expire [{}]", cacheMaxSize, cacheExpire); - this.cache = cacheBuilder.removalListener(new ScriptCacheRemovalListener()).build(); + Objects.requireNonNull(scriptContextRegistry); Map enginesByLangBuilder = new HashMap<>(); Map enginesByExtBuilder = new HashMap<>(); @@ -150,323 +98,157 @@ public ScriptService(Settings settings, Environment env, this.scriptEnginesByLang = unmodifiableMap(enginesByLangBuilder); this.scriptEnginesByExt = unmodifiableMap(enginesByExtBuilder); - this.scriptModes = new ScriptModes(scriptSettings, settings); - - // add file watcher for static scripts - scriptsDirectory = env.scriptsFile(); - if (logger.isTraceEnabled()) { - logger.trace("Using scripts directory [{}] ", scriptsDirectory); - } - FileWatcher fileWatcher = new FileWatcher(scriptsDirectory); - fileWatcher.addListener(new ScriptChangesListener()); - - if (SCRIPT_AUTO_RELOAD_ENABLED_SETTING.get(settings)) { - // automatic reload is enabled - register scripts - resourceWatcherService.add(fileWatcher); - } else { - // automatic reload is disable just load scripts once - fileWatcher.init(); - } - - this.lastInlineCompileTime = System.nanoTime(); - this.setMaxCompilationsPerMinute(SCRIPT_MAX_COMPILATIONS_PER_MINUTE.get(settings)); - } - - void registerClusterSettingsListeners(ClusterSettings clusterSettings) { - clusterSettings.addSettingsUpdateConsumer(SCRIPT_MAX_COMPILATIONS_PER_MINUTE, this::setMaxCompilationsPerMinute); - } - - @Override - public void close() throws IOException { - IOUtils.close(scriptEngines); - } - - private ScriptEngineService getScriptEngineServiceForLang(String lang) { - ScriptEngineService scriptEngineService = scriptEnginesByLang.get(lang); - if (scriptEngineService == null) { - throw new IllegalArgumentException("script_lang not supported [" + lang + "]"); - } - return scriptEngineService; - } - - private ScriptEngineService getScriptEngineServiceForFileExt(String fileExtension) { - ScriptEngineService scriptEngineService = scriptEnginesByExt.get(fileExtension); - if (scriptEngineService == null) { - throw new IllegalArgumentException("script file extension not supported [" + fileExtension + "]"); - } - return scriptEngineService; - } - - void setMaxCompilationsPerMinute(Integer newMaxPerMinute) { - this.totalCompilesPerMinute = newMaxPerMinute; - // Reset the counter to allow new compilations - this.scriptsPerMinCounter = totalCompilesPerMinute; - this.compilesAllowedPerNano = ((double) totalCompilesPerMinute) / TimeValue.timeValueMinutes(1).nanos(); - } - - /** - * Checks if a script can be executed and compiles it if needed, or returns the previously compiled and cached script. - */ - public CompiledScript compile(Script script, ScriptContext scriptContext) { - Objects.requireNonNull(script); - Objects.requireNonNull(scriptContext); - - ScriptType type = script.getType(); - String lang = script.getLang(); - String idOrCode = script.getIdOrCode(); - Map options = script.getOptions(); - - String id = idOrCode; - - // lang may be null when looking up a stored script, so we must get the - // source to retrieve the lang before checking if the context is supported - if (type == ScriptType.STORED) { - // search template requests can possibly pass in the entire path instead - // of just an id for looking up a stored script, so we parse the path and - // check for appropriate errors - String[] path = id.split("/"); - - if (path.length == 3) { - if (lang != null && lang.equals(path[1]) == false) { - throw new IllegalStateException("conflicting script languages, found [" + path[1] + "] but expected [" + lang + "]"); + this.scriptMetrics = scriptMetrics; + this.scriptPermits = new ScriptPermits(settings, scriptSettings, scriptContextRegistry); + this.compiler = new CachingCompiler(settings, scriptSettings, env, resourceWatcherService, scriptMetrics) { + @Override + protected CacheKey cacheKeyForFile(String baseName, String extension) { + if (extension.equals("mustache")) { + // For backwards compatibility mustache templates are in the scripts directory and we must ignore them here + return null; } + ScriptEngineService engine = scriptEnginesByExt.get(extension); + if (engine == null) { + logger.warn("script file extension not supported [" + extension + "]"); + return null; + } + return new CacheKey(engine.getType(), baseName, null); + } - id = path[2]; - - deprecationLogger.deprecated("use of [" + idOrCode + "] for looking up" + - " stored scripts/templates has been deprecated, use only [" + id + "] instead"); - } else if (path.length != 1) { - throw new IllegalArgumentException("illegal stored script format [" + id + "] use only "); + @Override + protected CacheKey cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { + return new CacheKey(scriptMetadata.getLang(), scriptMetadata.getCode(), scriptMetadata.getOptions()); } - // a stored script must be pulled from the cluster state every time in case - // the script has been updated since the last compilation - StoredScriptSource source = getScriptFromClusterState(id, lang); - lang = source.getLang(); - idOrCode = source.getCode(); - options = source.getOptions(); - } + @Override + protected void checkPutSupported(StoredScriptSource source) { + if (false == isLangSupported(source.getLang())) { + throw new IllegalArgumentException("unable to put stored script with unsupported lang [" + source.getLang() + "]"); + } + } - // TODO: fix this through some API or something, that's wrong - // special exception to prevent expressions from compiling as update or mapping scripts - boolean expression = "expression".equals(script.getLang()); - boolean notSupported = scriptContext.getKey().equals(ScriptContext.Standard.UPDATE.getKey()); - if (expression && notSupported) { - throw new UnsupportedOperationException("scripts of type [" + script.getType() + "]," + - " operation [" + scriptContext.getKey() + "] and lang [" + lang + "] are not supported"); - } + @Override + protected StoredScriptSource lookupStoredScript(ClusterState clusterState, CacheKey cacheKey) { + ScriptMetaData scriptMetadata = clusterState.metaData().custom(ScriptMetaData.TYPE); + if (scriptMetadata == null) { + return null; + } + + if (cacheKey.lang != null && isLangSupported(cacheKey.lang) == false) { + throw new IllegalArgumentException("unable to get stored script with unsupported lang [" + cacheKey.lang + "]"); + } - ScriptEngineService scriptEngineService = getScriptEngineServiceForLang(lang); + String id = cacheKey.idOrCode; + // search template requests can possibly pass in the entire path instead + // of just an id for looking up a stored script, so we parse the path and + // check for appropriate errors + String[] path = id.split("/"); - if (canExecuteScript(lang, type, scriptContext) == false) { - throw new IllegalStateException("scripts of type [" + script.getType() + "]," + - " operation [" + scriptContext.getKey() + "] and lang [" + lang + "] are disabled"); - } + if (path.length == 3) { + if (cacheKey.lang != null && cacheKey.lang.equals(path[1]) == false) { + throw new IllegalStateException("conflicting script languages, found [" + path[1] + + "] but expected [" + cacheKey.lang + "]"); + } - if (logger.isTraceEnabled()) { - logger.trace("compiling lang: [{}] type: [{}] script: {}", lang, type, idOrCode); - } + id = path[2]; - if (type == ScriptType.FILE) { - CacheKey cacheKey = new CacheKey(lang, idOrCode, options); - CompiledScript compiledScript = staticCache.get(cacheKey); + deprecationLogger.deprecated("use of [" + cacheKey.idOrCode + "] for looking up" + + " stored scripts/templates has been deprecated, use only [" + id + "] instead"); + } else if (path.length != 1) { + throw new IllegalArgumentException("illegal stored script format [" + id + "] use only "); + } - if (compiledScript == null) { - throw new IllegalArgumentException("unable to find file script [" + idOrCode + "] using lang [" + lang + "]"); + return scriptMetadata.getStoredScript(id, cacheKey.lang); } - return compiledScript; - } - - CacheKey cacheKey = new CacheKey(lang, idOrCode, options); - CompiledScript compiledScript = cache.get(cacheKey); - - if (compiledScript != null) { - return compiledScript; - } - - // Synchronize so we don't compile scripts many times during multiple shards all compiling a script - synchronized (this) { - // Retrieve it again in case it has been put by a different thread - compiledScript = cache.get(cacheKey); - - if (compiledScript == null) { - try { - // Either an un-cached inline script or indexed script - // If the script type is inline the name will be the same as the code for identification in exceptions - - // but give the script engine the chance to be better, give it separate name + source code - // for the inline case, then its anonymous: null. - if (logger.isTraceEnabled()) { - logger.trace("compiling script, type: [{}], lang: [{}], options: [{}]", type, lang, options); + @Override + protected boolean areAnyScriptContextsEnabled(CacheKey cacheKey, ScriptType scriptType) { + for (ScriptContext scriptContext : scriptContextRegistry.scriptContexts()) { + if (scriptPermits.canExecuteScript(cacheKey.lang, scriptType, scriptContext)) { + return true; } - // Check whether too many compilations have happened - checkCompilationLimit(); - compiledScript = new CompiledScript(type, id, lang, scriptEngineService.compile(id, idOrCode, options)); - } catch (ScriptException good) { - // TODO: remove this try-catch completely, when all script engines have good exceptions! - throw good; // its already good - } catch (Exception exception) { - throw new GeneralScriptException("Failed to compile " + type + " script [" + id + "] using lang [" + lang + "]", exception); } - - // Since the cache key is the script content itself we don't need to - // invalidate/check the cache if an indexed script changes. - scriptMetrics.onCompilation(); - cache.put(cacheKey, compiledScript); + return false; } - return compiledScript; - } - } - - /** - * Check whether there have been too many compilations within the last minute, throwing a circuit breaking exception if so. - * This is a variant of the token bucket algorithm: https://en.wikipedia.org/wiki/Token_bucket - * - * It can be thought of as a bucket with water, every time the bucket is checked, water is added proportional to the amount of time that - * elapsed since the last time it was checked. If there is enough water, some is removed and the request is allowed. If there is not - * enough water the request is denied. Just like a normal bucket, if water is added that overflows the bucket, the extra water/capacity - * is discarded - there can never be more water in the bucket than the size of the bucket. - */ - void checkCompilationLimit() { - long now = System.nanoTime(); - long timePassed = now - lastInlineCompileTime; - lastInlineCompileTime = now; - - scriptsPerMinCounter += (timePassed) * compilesAllowedPerNano; - - // It's been over the time limit anyway, readjust the bucket to be level - if (scriptsPerMinCounter > totalCompilesPerMinute) { - scriptsPerMinCounter = totalCompilesPerMinute; - } - - // If there is enough tokens in the bucket, allow the request and decrease the tokens by 1 - if (scriptsPerMinCounter >= 1) { - scriptsPerMinCounter -= 1.0; - } else { - // Otherwise reject the request - throw new CircuitBreakingException("[script] Too many dynamic script compilations within one minute, max: [" + - totalCompilesPerMinute + "/min]; please use on-disk, indexed, or scripts with parameters instead; " + - "this limit can be changed by the [" + SCRIPT_MAX_COMPILATIONS_PER_MINUTE.getKey() + "] setting"); - } - } - - public boolean isLangSupported(String lang) { - Objects.requireNonNull(lang); - - return scriptEnginesByLang.containsKey(lang); - } - - StoredScriptSource getScriptFromClusterState(String id, String lang) { - if (lang != null && isLangSupported(lang) == false) { - throw new IllegalArgumentException("unable to get stored script with unsupported lang [" + lang + "]"); - } - - ScriptMetaData scriptMetadata = clusterState.metaData().custom(ScriptMetaData.TYPE); - - if (scriptMetadata == null) { - throw new ResourceNotFoundException("unable to find script [" + id + "]" + - (lang == null ? "" : " using lang [" + lang + "]") + " in cluster state"); - } - - StoredScriptSource source = scriptMetadata.getStoredScript(id, lang); - - if (source == null) { - throw new ResourceNotFoundException("unable to find script [" + id + "]" + - (lang == null ? "" : " using lang [" + lang + "]") + " in cluster state"); - } - - return source; - } - - public void putStoredScript(ClusterService clusterService, PutStoredScriptRequest request, - ActionListener listener) { - int max = SCRIPT_MAX_SIZE_IN_BYTES.get(settings); - - if (request.content().length() > max) { - throw new IllegalArgumentException("exceeded max allowed stored script size in bytes [" + max + "] with size [" + - request.content().length() + "] for script [" + request.id() + "]"); - } - - StoredScriptSource source = StoredScriptSource.parse(request.lang(), request.content(), request.xContentType()); - - if (isLangSupported(source.getLang()) == false) { - throw new IllegalArgumentException("unable to put stored script with unsupported lang [" + source.getLang() + "]"); - } - - try { - ScriptEngineService scriptEngineService = getScriptEngineServiceForLang(source.getLang()); - - if (isAnyScriptContextEnabled(source.getLang(), ScriptType.STORED)) { - Object compiled = scriptEngineService.compile(request.id(), source.getCode(), Collections.emptyMap()); - - if (compiled == null) { - throw new IllegalArgumentException("failed to parse/compile stored script [" + request.id() + "]" + - (source.getCode() == null ? "" : " using code [" + source.getCode() + "]")); + @Override + protected void checkCanExecuteScript(CacheKey cacheKey, ScriptType scriptType, ScriptContext scriptContext) { + if (isLangSupported(cacheKey.lang) == false) { + throw new IllegalArgumentException("script_lang not supported [" + cacheKey.lang + "]"); + } + // TODO: fix this through some API or something, this is a silly way to do this + // special exception to prevent expressions from compiling as update or mapping scripts + boolean expression = "expression".equals(cacheKey.lang); + boolean notSupported = scriptContext.getKey().equals(ScriptContext.Standard.UPDATE.getKey()); + if (expression && notSupported) { + throw new UnsupportedOperationException("scripts of type [" + scriptType + "]," + + " operation [" + scriptContext.getKey() + "] and lang [" + cacheKey.lang + "] are not supported"); + } + if (scriptPermits.canExecuteScript(cacheKey.lang, scriptType, scriptContext) == false) { + throw new IllegalStateException("scripts of type [" + scriptType + "]," + + " operation [" + scriptContext.getKey() + "] and lang [" + cacheKey.lang + "] are disabled"); } - } else { - throw new IllegalArgumentException( - "cannot put stored script [" + request.id() + "], stored scripts cannot be run under any context"); } - } catch (ScriptException good) { - throw good; - } catch (Exception exception) { - throw new IllegalArgumentException("failed to parse/compile stored script [" + request.id() + "]", exception); - } - clusterService.submitStateUpdateTask("put-script-" + request.id(), - new AckedClusterStateUpdateTask(request, listener) { + @Override + protected void checkCompilationLimit() { + scriptPermits.checkCompilationLimit(); + } @Override - protected PutStoredScriptResponse newResponse(boolean acknowledged) { - return new PutStoredScriptResponse(acknowledged); + protected CompiledScript compile(ScriptType scriptType, CacheKey cacheKey) { + return compile(scriptType, cacheKey, cacheKey.idOrCode, null); } @Override - public ClusterState execute(ClusterState currentState) throws Exception { - ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); - smd = ScriptMetaData.putStoredScript(smd, request.id(), source); - MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); + protected CompiledScript compileFileScript(CacheKey cacheKey, String body, Path file) { + // pass the actual file name to the compiler (for script engines that care about this) + return compile(ScriptType.FILE, cacheKey, body, file.getFileName().toString()); + } - return ClusterState.builder(currentState).metaData(mdb).build(); + private CompiledScript compile(ScriptType scriptType, CacheKey cacheKey, String body, String fileName) { + ScriptEngineService engine = scriptEnginesByLang.get(cacheKey.lang); + if (engine == null) { + throw new IllegalArgumentException("script lang not supported [" + cacheKey.lang + "]"); + } + Object executable = engine.compile(fileName, body, cacheKey.options); + return new CompiledScript(scriptType, body, engine.getType(), executable); } - }); + }; } - public void deleteStoredScript(ClusterService clusterService, DeleteStoredScriptRequest request, - ActionListener listener) { - if (request.lang() != null && isLangSupported(request.lang()) == false) { - throw new IllegalArgumentException("unable to delete stored script with unsupported lang [" + request.lang() +"]"); - } + @Override + public void close() throws IOException { + IOUtils.close(scriptEngines); + } - clusterService.submitStateUpdateTask("delete-script-" + request.id(), - new AckedClusterStateUpdateTask(request, listener) { + public void registerClusterSettingsListeners(ClusterSettings clusterSettings) { + scriptPermits.registerClusterSettingsListeners(clusterSettings); + } - @Override - protected DeleteStoredScriptResponse newResponse(boolean acknowledged) { - return new DeleteStoredScriptResponse(acknowledged); - } + private ScriptEngineService getScriptEngineServiceForLang(String lang) { + ScriptEngineService scriptEngineService = scriptEnginesByLang.get(lang); + if (scriptEngineService == null) { + throw new IllegalArgumentException("script_lang not supported [" + lang + "]"); + } + return scriptEngineService; + } - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); - smd = ScriptMetaData.deleteStoredScript(smd, request.id(), request.lang()); - MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); + /** + * Checks if a script can be executed and compiles it if needed, or returns the previously compiled and cached script. + */ + public CompiledScript compile(Script script, ScriptContext scriptContext) { + Objects.requireNonNull(script); + Objects.requireNonNull(scriptContext); - return ClusterState.builder(currentState).metaData(mdb).build(); - } - }); + CacheKey cacheKey = new CacheKey(script.getLang(), script.getIdOrCode(), script.getOptions()); + return compiler.getScript(cacheKey, script.getType(), scriptContext); } - public StoredScriptSource getStoredScript(ClusterState state, GetStoredScriptRequest request) { - ScriptMetaData scriptMetadata = state.metaData().custom(ScriptMetaData.TYPE); + public boolean isLangSupported(String lang) { + Objects.requireNonNull(lang); - if (scriptMetadata != null) { - return scriptMetadata.getStoredScript(request.id(), request.lang()); - } else { - return null; - } + return scriptEnginesByLang.containsKey(lang); } /** @@ -499,141 +281,31 @@ public SearchScript search(SearchLookup lookup, CompiledScript compiledScript, return getScriptEngineServiceForLang(compiledScript.lang()).search(compiledScript, lookup, params); } - private boolean isAnyScriptContextEnabled(String lang, ScriptType scriptType) { - for (ScriptContext scriptContext : scriptContextRegistry.scriptContexts()) { - if (canExecuteScript(lang, scriptType, scriptContext)) { - return true; - } - } - return false; - } - - private boolean canExecuteScript(String lang, ScriptType scriptType, ScriptContext scriptContext) { - assert lang != null; - if (scriptContextRegistry.isSupportedContext(scriptContext) == false) { - throw new IllegalArgumentException("script context [" + scriptContext.getKey() + "] not supported"); - } - return scriptModes.getScriptEnabled(lang, scriptType, scriptContext); - } - public ScriptStats stats() { return scriptMetrics.stats(); } @Override public void clusterChanged(ClusterChangedEvent event) { - clusterState = event.state(); + compiler.clusterChanged(event); } - /** - * A small listener for the script cache that calls each - * {@code ScriptEngineService}'s {@code scriptRemoved} method when the - * script has been removed from the cache - */ - private class ScriptCacheRemovalListener implements RemovalListener { - @Override - public void onRemoval(RemovalNotification notification) { - if (logger.isDebugEnabled()) { - logger.debug("removed {} from cache, reason: {}", notification.getValue(), notification.getRemovalReason()); - } - scriptMetrics.onCacheEviction(); - } + public final StoredScriptSource getStoredScript(ClusterState state, GetStoredScriptRequest request) { + return compiler.getStoredScript(state, request); } - private class ScriptChangesListener implements FileChangesListener { - - private Tuple getScriptNameExt(Path file) { - Path scriptPath = scriptsDirectory.relativize(file); - int extIndex = scriptPath.toString().lastIndexOf('.'); - if (extIndex <= 0) { - return null; - } - - String ext = scriptPath.toString().substring(extIndex + 1); - if (ext.isEmpty()) { - return null; - } - - String scriptName = scriptPath.toString().substring(0, extIndex).replace(scriptPath.getFileSystem().getSeparator(), "_"); - return new Tuple<>(scriptName, ext); - } - - @Override - public void onFileInit(Path file) { - Tuple scriptNameExt = getScriptNameExt(file); - if (scriptNameExt == null) { - logger.debug("Skipped script with invalid extension : [{}]", file); - return; - } - if (logger.isTraceEnabled()) { - logger.trace("Loading script file : [{}]", file); - } - - ScriptEngineService engineService = getScriptEngineServiceForFileExt(scriptNameExt.v2()); - if (engineService == null) { - logger.warn("No script engine found for [{}]", scriptNameExt.v2()); - } else { - try { - //we don't know yet what the script will be used for, but if all of the operations for this lang - // with file scripts are disabled, it makes no sense to even compile it and cache it. - if (isAnyScriptContextEnabled(engineService.getType(), ScriptType.FILE)) { - logger.info("compiling script file [{}]", file.toAbsolutePath()); - try (InputStreamReader reader = new InputStreamReader(Files.newInputStream(file), StandardCharsets.UTF_8)) { - String script = Streams.copyToString(reader); - String id = scriptNameExt.v1(); - CacheKey cacheKey = new CacheKey(engineService.getType(), id, null); - // pass the actual file name to the compiler (for script engines that care about this) - Object executable = engineService.compile(file.getFileName().toString(), script, Collections.emptyMap()); - CompiledScript compiledScript = new CompiledScript(ScriptType.FILE, id, engineService.getType(), executable); - staticCache.put(cacheKey, compiledScript); - scriptMetrics.onCompilation(); - } - } else { - logger.warn("skipping compile of script file [{}] as all scripted operations are disabled for file scripts", file.toAbsolutePath()); - } - } catch (ScriptException e) { - try (XContentBuilder builder = JsonXContent.contentBuilder()) { - builder.prettyPrint(); - builder.startObject(); - ElasticsearchException.generateThrowableXContent(builder, ToXContent.EMPTY_PARAMS, e); - builder.endObject(); - logger.warn("failed to load/compile script [{}]: {}", scriptNameExt.v1(), builder.string()); - } catch (IOException ioe) { - ioe.addSuppressed(e); - logger.warn((Supplier) () -> new ParameterizedMessage( - "failed to log an appropriate warning after failing to load/compile script [{}]", scriptNameExt.v1()), ioe); - } - /* Log at the whole exception at the debug level as well just in case the stack trace is important. That way you can - * turn on the stack trace if you need it. */ - logger.debug((Supplier) () -> new ParameterizedMessage("failed to load/compile script [{}]. full exception:", - scriptNameExt.v1()), e); - } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("failed to load/compile script [{}]", scriptNameExt.v1()), e); - } - } - } - - @Override - public void onFileCreated(Path file) { - onFileInit(file); - } - - @Override - public void onFileDeleted(Path file) { - Tuple scriptNameExt = getScriptNameExt(file); - if (scriptNameExt != null) { - ScriptEngineService engineService = getScriptEngineServiceForFileExt(scriptNameExt.v2()); - assert engineService != null; - logger.info("removing script file [{}]", file.toAbsolutePath()); - staticCache.remove(new CacheKey(engineService.getType(), scriptNameExt.v1(), null)); - } - } + public void putStoredScript(ClusterService clusterService, PutStoredScriptRequest request, + ActionListener listener) { + compiler.putStoredScript(clusterService, request, listener); + } - @Override - public void onFileChanged(Path file) { - onFileInit(file); + public void deleteStoredScript(ClusterService clusterService, DeleteStoredScriptRequest request, + ActionListener listener) { + if (request.lang() != null && isLangSupported(request.lang()) == false) { + throw new IllegalArgumentException("unable to delete stored script with unsupported lang [" + request.lang() + "]"); } + compiler.deleteStoredScript(clusterService, request, listener); } private static final class CacheKey { @@ -667,5 +339,14 @@ public int hashCode() { result = 31 * result + (options != null ? options.hashCode() : 0); return result; } + + @Override + public String toString() { + String result = "[lang=" + lang + ", id=" + idOrCode; + if (options != null) { + result += ", options " + options; + } + return result + "]"; + } } } diff --git a/core/src/main/java/org/elasticsearch/script/ScriptSettings.java b/core/src/main/java/org/elasticsearch/script/ScriptSettings.java index 447097a488404..5ee18862b3e99 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptSettings.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptSettings.java @@ -52,7 +52,8 @@ public ScriptSettings(ScriptEngineRegistry scriptEngineRegistry, ScriptContextRe Map> scriptContextSettingMap = contextSettings(scriptContextRegistry); this.scriptContextSettingMap = Collections.unmodifiableMap(scriptContextSettingMap); - List> scriptLanguageSettings = languageSettings(SCRIPT_TYPE_SETTING_MAP, scriptContextSettingMap, scriptEngineRegistry, scriptContextRegistry); + List> scriptLanguageSettings = languageSettings(SCRIPT_TYPE_SETTING_MAP, scriptContextSettingMap, + scriptEngineRegistry, scriptContextRegistry); this.scriptLanguageSettings = Collections.unmodifiableList(scriptLanguageSettings); } diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java new file mode 100644 index 0000000000000..5dfcac527b448 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -0,0 +1,141 @@ +/* + * 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.apache.logging.log4j.Logger; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.common.logging.DeprecationLogger; +import org.elasticsearch.common.logging.ESLoggerFactory; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.watcher.ResourceWatcherService; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.Map; +import java.util.Objects; + +import static java.util.Collections.emptyMap; + +public class TemplateService { + public interface Backend extends ScriptEngineService {} // NOCOMMIT this should diverge.... + + private static final Logger logger = ESLoggerFactory.getLogger(TemplateService.class); + private static final DeprecationLogger deprecationLogger = new DeprecationLogger(logger); + private final Backend backend; + private final ScriptPermits scriptPermits; + private final CachingCompiler compiler; + + public TemplateService(Settings settings, Environment env, ResourceWatcherService resourceWatcherService, + Backend backend, ScriptContextRegistry scriptContextRegistry, ScriptSettings scriptSettings, + ScriptMetrics scriptMetrics) throws IOException { + Objects.requireNonNull(scriptContextRegistry); + + this.backend = backend; + this.scriptPermits = new ScriptPermits(settings, scriptSettings, scriptContextRegistry); + this.compiler = new CachingCompiler(settings, scriptSettings, env, resourceWatcherService, scriptMetrics) { + @Override + protected String cacheKeyForFile(String baseName, String extension) { + if (false == backend.getType().equals(extension)) { + // For backwards compatibility templates are in the scripts directory and we must ignore all other templates + return null; + } + return baseName; + } + + @Override + protected String cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { + return scriptMetadata.getCode(); + } + + @Override + protected void checkPutSupported(StoredScriptSource source) { + if (false == backend.getType().equals(source.getLang())) { + throw new IllegalArgumentException("only mustache scripts are supported here but got [" + source.getLang() + "]"); + } + } + + @Override + protected StoredScriptSource lookupStoredScript(ClusterState clusterState, String cacheKey) { + ScriptMetaData scriptMetadata = clusterState.metaData().custom(ScriptMetaData.TYPE); + if (scriptMetadata == null) { + return null; + } + + String id = cacheKey; + // search template requests can possibly pass in the entire path instead + // of just an id for looking up a stored script, so we parse the path and + // check for appropriate errors + String[] path = id.split("/"); + + if (path.length == 3) { + id = path[2]; + + deprecationLogger.deprecated("use of [" + cacheKey + "] for looking up" + + " stored scripts/templates has been deprecated, use only [" + id + "] instead"); + } else if (path.length != 1) { + throw new IllegalArgumentException("illegal stored script format [" + id + "] use only "); + } + + return scriptMetadata.getStoredScript(id, "mustache"); + } + + @Override + protected boolean areAnyScriptContextsEnabled(String cacheKey, ScriptType scriptType) { + for (ScriptContext scriptContext : scriptContextRegistry.scriptContexts()) { + if (scriptPermits.canExecuteScript(backend.getType(), scriptType, scriptContext)) { + return true; + } + } + return false; + } + + @Override + protected void checkCanExecuteScript(String cacheKey, ScriptType scriptType, ScriptContext scriptContext) { + if (scriptPermits.canExecuteScript(backend.getType(), scriptType, scriptContext) == false) { + throw new IllegalStateException("scripts of type [" + scriptType + "]," + + " operation [" + scriptContext.getKey() + "] and lang [" + backend.getType() + "] are disabled"); + } + } + + @Override + protected void checkCompilationLimit() { + scriptPermits.checkCompilationLimit(); + } + + @Override + protected CompiledScript compile(ScriptType scriptType, String cacheKey) { + Object compiled = backend.compile(null, cacheKey, emptyMap()); + return new CompiledScript(scriptType, cacheKey, backend.getType(), compiled); + } + + @Override + protected CompiledScript compileFileScript(String cacheKey, String body, Path file) { + Object compiled = backend.compile(file.getFileName().toString(), cacheKey, emptyMap()); + return new CompiledScript(ScriptType.FILE, body, backend.getType(), compiled); + } + }; + } + + public ExecutableScript executable(String idOrCode, ScriptType scriptType, ScriptContext scriptContext, + Map scriptParams) { + return backend.executable(compiler.getScript(idOrCode, scriptType, scriptContext), scriptParams); + } +} diff --git a/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java b/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java index 4b11697c16d9f..a731ebee615fa 100644 --- a/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java @@ -37,6 +37,7 @@ import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptContextRegistry; import org.elasticsearch.script.ScriptEngineRegistry; +import org.elasticsearch.script.ScriptMetrics; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptSettings; import org.elasticsearch.script.ScriptType; @@ -285,8 +286,8 @@ public void testNowInScript() throws IOException { scripts))); ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); - ScriptService scriptService = new ScriptService(baseSettings, environment, - new ResourceWatcherService(baseSettings, null), scriptEngineRegistry, scriptContextRegistry, scriptSettings); + ScriptService scriptService = new ScriptService(baseSettings, environment, new ResourceWatcherService(baseSettings, null), + scriptEngineRegistry, scriptContextRegistry, scriptSettings, new ScriptMetrics()); Settings settings = settings(Version.CURRENT).build(); UpdateHelper updateHelper = new UpdateHelper(settings, scriptService); diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 106dc9274da93..42d1d6ab3662a 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -68,10 +68,10 @@ import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.script.ScriptContextRegistry; import org.elasticsearch.script.ScriptEngineRegistry; +import org.elasticsearch.script.ScriptMetrics; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptSettings; import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.TestSearchContext; @@ -129,8 +129,7 @@ public void setUp() throws Exception { ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); scriptService = new ScriptService(settings, environment, new ResourceWatcherService(settings, threadPool), scriptEngineRegistry, - scriptContextRegistry, scriptSettings); - clusterService = ClusterServiceUtils.createClusterService(threadPool); + scriptContextRegistry, scriptSettings, new ScriptMetrics()); nodeEnvironment = new NodeEnvironment(settings, environment); mapperRegistry = new IndicesModule(Collections.emptyList()).getMapperRegistry(); } @@ -138,13 +137,13 @@ public void setUp() throws Exception { @Override public void tearDown() throws Exception { super.tearDown(); - IOUtils.close(nodeEnvironment, indicesQueryCache, clusterService); + IOUtils.close(nodeEnvironment, indicesQueryCache); ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); } private IndexService newIndexService(IndexModule module) throws IOException { return module.newIndexService(nodeEnvironment, xContentRegistry(), deleter, circuitBreakerService, bigArrays, threadPool, - scriptService, clusterService, null, indicesQueryCache, mapperRegistry, shardId -> {}, + scriptService, null, null, indicesQueryCache, mapperRegistry, shardId -> {}, new IndicesFieldDataCache(settings, listener)); } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java b/core/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java index f6a224e8a4b3d..5c88e28617eea 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java @@ -74,7 +74,7 @@ public void modify(MappedFieldType ft) { } public void testIsFieldWithinQueryEmptyReader() throws IOException { - QueryRewriteContext context = new QueryRewriteContext(null, null, null, xContentRegistry(), null, null, + QueryRewriteContext context = new QueryRewriteContext(null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis); IndexReader reader = new MultiReader(); DateFieldType ft = new DateFieldType(); @@ -85,7 +85,7 @@ public void testIsFieldWithinQueryEmptyReader() throws IOException { private void doTestIsFieldWithinQuery(DateFieldType ft, DirectoryReader reader, DateTimeZone zone, DateMathParser alternateFormat) throws IOException { - QueryRewriteContext context = new QueryRewriteContext(null, null, null, xContentRegistry(), null, null, + QueryRewriteContext context = new QueryRewriteContext(null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis); assertEquals(Relation.INTERSECTS, ft.isFieldWithinQuery(reader, "2015-10-09", "2016-01-02", randomBoolean(), randomBoolean(), null, null, context)); @@ -133,7 +133,7 @@ public void testIsFieldWithinQuery() throws IOException { DateFieldType ft2 = new DateFieldType(); ft2.setName("my_date2"); - QueryRewriteContext context = new QueryRewriteContext(null, null, null, xContentRegistry(), null, null, + QueryRewriteContext context = new QueryRewriteContext(null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis); assertEquals(Relation.DISJOINT, ft2.isFieldWithinQuery(reader, "2015-10-09", "2016-01-02", false, false, null, null, context)); IOUtils.close(reader, w, dir); @@ -167,9 +167,8 @@ public void testTermQuery() { Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).build(); QueryShardContext context = new QueryShardContext(0, - new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), - indexSettings), - null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis); + new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings), + null, null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis); MappedFieldType ft = createDefaultFieldType(); ft.setName("field"); String date = "2015-10-12T14:10:55"; @@ -191,7 +190,7 @@ public void testRangeQuery() throws IOException { .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).build(); QueryShardContext context = new QueryShardContext(0, new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings), - null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis); + null, null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis); MappedFieldType ft = createDefaultFieldType(); ft.setName("field"); String date1 = "2015-10-12T14:10:55"; diff --git a/core/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java b/core/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java index f70db120fcfc3..7d52d33016ce1 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java @@ -74,7 +74,7 @@ public void testRangeQuery() throws Exception { Settings indexSettings = Settings.builder() .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAsciiOfLengthBetween(1, 10), indexSettings); - QueryShardContext context = new QueryShardContext(0, idxSettings, null, null, null, null, null, xContentRegistry(), + QueryShardContext context = new QueryShardContext(0, idxSettings, null, null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis); RangeFieldMapper.RangeFieldType ft = new RangeFieldMapper.RangeFieldType(type); ft.setName(FIELDNAME); diff --git a/core/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java b/core/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java index b83c1b2897a3c..77b7549e98aa4 100644 --- a/core/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java @@ -48,7 +48,7 @@ public void testFailIfFieldMappingNotFound() { when(mapperService.getIndexSettings()).thenReturn(indexSettings); final long nowInMillis = randomNonNegativeLong(); QueryShardContext context = new QueryShardContext( - 0, indexSettings, null, null, mapperService, null, null, xContentRegistry(), null, null, + 0, indexSettings, null, null, mapperService, null, null, null, xContentRegistry(), null, null, () -> nowInMillis); context.setAllowUnmappedFields(false); diff --git a/core/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java b/core/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java index d7ef534bcb2bd..284f902178716 100644 --- a/core/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java @@ -37,7 +37,7 @@ public void testRewriteMissingField() throws Exception { IndexService indexService = createIndex("test"); IndexReader reader = new MultiReader(); QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, indexService.mapperService(), - null, null, xContentRegistry(), null, reader, null); + null, null, null, xContentRegistry(), null, reader, null); RangeQueryBuilder range = new RangeQueryBuilder("foo"); assertEquals(Relation.DISJOINT, range.getRelation(context)); } @@ -54,7 +54,7 @@ public void testRewriteMissingReader() throws Exception { indexService.mapperService().merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE, false); QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, indexService.mapperService(), - null, null, xContentRegistry(), null, null, null); + null, null, null, xContentRegistry(), null, null, null); RangeQueryBuilder range = new RangeQueryBuilder("foo"); // can't make assumptions on a missing reader, so it must return INTERSECT assertEquals(Relation.INTERSECTS, range.getRelation(context)); @@ -73,7 +73,7 @@ public void testRewriteEmptyReader() throws Exception { new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE, false); IndexReader reader = new MultiReader(); QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, indexService.mapperService(), - null, null, xContentRegistry(), null, reader, null); + null, null, null, xContentRegistry(), null, reader, null); RangeQueryBuilder range = new RangeQueryBuilder("foo"); // no values -> DISJOINT assertEquals(Relation.DISJOINT, range.getRelation(context)); diff --git a/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java b/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java index de423db442a1b..6f3cea722867d 100644 --- a/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/SimpleQueryParserTests.java @@ -177,7 +177,7 @@ public void testQuoteFieldSuffix() { .build(); IndexMetaData indexState = IndexMetaData.builder("index").settings(indexSettings).build(); IndexSettings settings = new IndexSettings(indexState, Settings.EMPTY); - QueryShardContext mockShardContext = new QueryShardContext(0, settings, null, null, null, null, null, xContentRegistry(), + QueryShardContext mockShardContext = new QueryShardContext(0, settings, null, null, null, null, null, null, xContentRegistry(), null, null, System::currentTimeMillis) { @Override public MappedFieldType fieldMapper(String name) { @@ -191,7 +191,7 @@ public MappedFieldType fieldMapper(String name) { assertEquals(new TermQuery(new Term("foo.quote", "bar")), parser.parse("\"bar\"")); // Now check what happens if foo.quote does not exist - mockShardContext = new QueryShardContext(0, settings, null, null, null, null, null, xContentRegistry(), + mockShardContext = new QueryShardContext(0, settings, null, null, null, null, null, null, xContentRegistry(), null, null, System::currentTimeMillis) { @Override public MappedFieldType fieldMapper(String name) { diff --git a/core/src/test/java/org/elasticsearch/script/FileScriptTests.java b/core/src/test/java/org/elasticsearch/script/FileScriptTests.java index 92e659ac755d5..57d6d6b8f192b 100644 --- a/core/src/test/java/org/elasticsearch/script/FileScriptTests.java +++ b/core/src/test/java/org/elasticsearch/script/FileScriptTests.java @@ -47,7 +47,8 @@ ScriptService makeScriptService(Settings settings) throws Exception { ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singleton(scriptEngine)); ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); - return new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, scriptContextRegistry, scriptSettings); + return new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, scriptContextRegistry, scriptSettings, + new ScriptMetrics()); } public void testFileScriptFound() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java b/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java index bf5c7e0daa782..64f5378ee07a7 100644 --- a/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java +++ b/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java @@ -19,13 +19,6 @@ package org.elasticsearch.script; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; @@ -34,6 +27,13 @@ import org.elasticsearch.test.InternalSettingsPlugin; import org.elasticsearch.watcher.ResourceWatcherService; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; import static java.util.Collections.singletonMap; @@ -48,7 +48,8 @@ public void testNativeScript() throws InterruptedException { .put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), false) .build(); ScriptModule scriptModule = new ScriptModule(settings, new Environment(settings), null, - singletonList(new NativeScriptEngineService(settings, singletonMap("my", new MyNativeScriptFactory()))), emptyList()); + singletonList(new NativeScriptEngineService(settings, singletonMap("my", new MyNativeScriptFactory()))), emptyList(), + null); List> scriptSettings = scriptModule.getSettings(); scriptSettings.add(InternalSettingsPlugin.VERSION_CREATED); @@ -76,7 +77,7 @@ public void testFineGrainedSettingsDontAffectNativeScripts() throws IOException ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(new ArrayList<>()); ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); ScriptService scriptService = new ScriptService(settings, environment, resourceWatcherService, scriptEngineRegistry, - scriptContextRegistry, scriptSettings); + scriptContextRegistry, scriptSettings, new ScriptMetrics()); for (ScriptContext scriptContext : scriptContextRegistry.scriptContexts()) { assertThat(scriptService.compile(new Script(ScriptType.INLINE, NativeScriptEngineService.NAME, "my", Collections.emptyMap()), diff --git a/core/src/test/java/org/elasticsearch/script/ScriptContextTests.java b/core/src/test/java/org/elasticsearch/script/ScriptContextTests.java index e25335e5d68e4..855d35ea7d84b 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptContextTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptContextTests.java @@ -54,7 +54,8 @@ ScriptService makeScriptService() throws Exception { new ScriptContext.Plugin(PLUGIN_NAME, "custom_globally_disabled_op")); ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(customContexts); ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); - ScriptService scriptService = new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, scriptContextRegistry, scriptSettings); + ScriptService scriptService = new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, + scriptContextRegistry, scriptSettings, new ScriptMetrics()); ClusterState empty = ClusterState.builder(new ClusterName("_name")).build(); ScriptMetaData smd = empty.metaData().custom(ScriptMetaData.TYPE); diff --git a/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java b/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java new file mode 100644 index 0000000000000..38b9e29121d93 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java @@ -0,0 +1,54 @@ +/* + * 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.breaker.CircuitBreakingException; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ESTestCase; + +import static java.util.Collections.emptyList; + +public class ScriptPermitsTests extends ESTestCase { + public void testCompilationCircuitBreaking() throws Exception { + ScriptContextRegistry contextRegistry = new ScriptContextRegistry(emptyList()); + ScriptSettings scriptSettings = new ScriptSettings(new ScriptEngineRegistry(emptyList()), contextRegistry); + ScriptPermits permits = new ScriptPermits(Settings.EMPTY, scriptSettings, contextRegistry); + permits.setMaxCompilationsPerMinute(1); + permits.checkCompilationLimit(); // should pass + expectThrows(CircuitBreakingException.class, () -> permits.checkCompilationLimit()); + permits.setMaxCompilationsPerMinute(2); + permits.checkCompilationLimit(); // should pass + permits.checkCompilationLimit(); // should pass + expectThrows(CircuitBreakingException.class, () -> permits.checkCompilationLimit()); + int count = randomIntBetween(5, 50); + permits.setMaxCompilationsPerMinute(count); + for (int i = 0; i < count; i++) { + permits.checkCompilationLimit(); // should pass + } + expectThrows(CircuitBreakingException.class, () -> permits.checkCompilationLimit()); + permits.setMaxCompilationsPerMinute(0); + expectThrows(CircuitBreakingException.class, () -> permits.checkCompilationLimit()); + permits.setMaxCompilationsPerMinute(Integer.MAX_VALUE); + int largeLimit = randomIntBetween(1000, 10000); + for (int i = 0; i < largeLimit; i++) { + permits.checkCompilationLimit(); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java b/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java index 3482dc8bb348b..8634bb9e323f4 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java @@ -18,15 +18,19 @@ */ package org.elasticsearch.script; +import com.carrotsearch.randomizedtesting.annotations.Repeat; + import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptRequest; +import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; @@ -51,7 +55,6 @@ import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.sameInstance; -//TODO: this needs to be a base test class, and all scripting engines extend it public class ScriptServiceTests extends ESTestCase { private ResourceWatcherService resourceWatcherService; @@ -67,7 +70,6 @@ public class ScriptServiceTests extends ESTestCase { private Settings baseSettings; private static final Map DEFAULT_SCRIPT_ENABLED = new HashMap<>(); - static { DEFAULT_SCRIPT_ENABLED.put(ScriptType.FILE, true); DEFAULT_SCRIPT_ENABLED.put(ScriptType.STORED, false); @@ -117,38 +119,8 @@ public void setup() throws IOException { private void buildScriptService(Settings additionalSettings) throws IOException { Settings finalSettings = Settings.builder().put(baseSettings).put(additionalSettings).build(); Environment environment = new Environment(finalSettings); - // TODO: - scriptService = new ScriptService(finalSettings, environment, resourceWatcherService, scriptEngineRegistry, scriptContextRegistry, scriptSettings) { - @Override - StoredScriptSource getScriptFromClusterState(String id, String lang) { - //mock the script that gets retrieved from an index - return new StoredScriptSource(lang, "100", Collections.emptyMap()); - } - }; - } - - public void testCompilationCircuitBreaking() throws Exception { - buildScriptService(Settings.EMPTY); - scriptService.setMaxCompilationsPerMinute(1); - scriptService.checkCompilationLimit(); // should pass - expectThrows(CircuitBreakingException.class, () -> scriptService.checkCompilationLimit()); - scriptService.setMaxCompilationsPerMinute(2); - scriptService.checkCompilationLimit(); // should pass - scriptService.checkCompilationLimit(); // should pass - expectThrows(CircuitBreakingException.class, () -> scriptService.checkCompilationLimit()); - int count = randomIntBetween(5, 50); - scriptService.setMaxCompilationsPerMinute(count); - for (int i = 0; i < count; i++) { - scriptService.checkCompilationLimit(); // should pass - } - expectThrows(CircuitBreakingException.class, () -> scriptService.checkCompilationLimit()); - scriptService.setMaxCompilationsPerMinute(0); - expectThrows(CircuitBreakingException.class, () -> scriptService.checkCompilationLimit()); - scriptService.setMaxCompilationsPerMinute(Integer.MAX_VALUE); - int largeLimit = randomIntBetween(1000, 10000); - for (int i = 0; i < largeLimit; i++) { - scriptService.checkCompilationLimit(); - } + scriptService = new ScriptService(finalSettings, environment, resourceWatcherService, scriptEngineRegistry, scriptContextRegistry, + scriptSettings, new ScriptMetrics()); } public void testNotSupportedDisableDynamicSetting() throws IOException { @@ -180,7 +152,7 @@ public void testScriptsWithoutExtensions() throws IOException { scriptService.compile(new Script(ScriptType.FILE, "test", "test_script", Collections.emptyMap()), ScriptContext.Standard.SEARCH); fail("the script test_script should no longer exist"); } catch (IllegalArgumentException ex) { - assertThat(ex.getMessage(), containsString("unable to find file script [test_script] using lang [test]")); + assertThat(ex.getMessage(), containsString("unable to find file script [lang=test, id=test_script]")); } } @@ -219,7 +191,12 @@ public void testDefaultBehaviourFineGrainedSettings() throws IOException { builder.put("script.file", "true"); } buildScriptService(builder.build()); + + // Setup file and cluster state scripts createFileScripts("mustache", "dtest"); + ClusterState state = stateWithScripts(new Tuple<>("script", + StoredScriptSource.parse("dtest", new BytesArray("{\"script\":\"abc\"}"), XContentType.JSON))); + scriptService.clusterChanged(new ClusterChangedEvent("test", state, stateWithScripts())); for (ScriptContext scriptContext : scriptContexts) { // only file scripts are accepted by default @@ -291,6 +268,9 @@ public void testFineGrainedSettings() throws IOException { buildScriptService(builder.build()); createFileScripts("expression", "mustache", "dtest"); + ClusterState state = stateWithScripts(new Tuple<>("script", StoredScriptSource.parse(dangerousScriptEngineService.getType(), + new BytesArray("{\"script\":\"abc\"}"), XContentType.JSON))); + scriptService.clusterChanged(new ClusterChangedEvent("test", state, stateWithScripts())); for (ScriptType scriptType : ScriptType.values()) { //make sure file scripts have a different name than inline ones. @@ -321,6 +301,7 @@ public void testFineGrainedSettings() throws IOException { public void testCompileNonRegisteredContext() throws IOException { buildScriptService(Settings.EMPTY); + String pluginName; String unknownContext; do { @@ -330,8 +311,13 @@ public void testCompileNonRegisteredContext() throws IOException { String type = scriptEngineService.getType(); try { - scriptService.compile(new Script(randomFrom(ScriptType.values()), type, "test", Collections.emptyMap()), - new ScriptContext.Plugin(pluginName, unknownContext)); + Script script = new Script(randomFrom(ScriptType.values()), type, "test", Collections.emptyMap()); + if (script.getType() == ScriptType.STORED) { + ClusterState state = stateWithScripts(new Tuple<>(script.getIdOrCode(), + StoredScriptSource.parse(script.getLang(), new BytesArray("{\"script\":\"abc\"}"), XContentType.JSON))); + scriptService.clusterChanged(new ClusterChangedEvent("test", state, stateWithScripts())); + } + scriptService.compile(script, new ScriptContext.Plugin(pluginName, unknownContext)); fail("script compilation should have been rejected"); } catch(IllegalArgumentException e) { assertThat(e.getMessage(), containsString("script context [" + pluginName + "_" + unknownContext + "] not supported")); @@ -385,6 +371,10 @@ public void testFileScriptCountedInCompilationStats() throws IOException { public void testIndexedScriptCountedInCompilationStats() throws IOException { buildScriptService(Settings.EMPTY); + ClusterState cs = stateWithScripts( + new Tuple<>("script", StoredScriptSource.parse("test", new BytesArray("{\"script\":\"abc\"}"), XContentType.JSON))); + scriptService.clusterChanged(new ClusterChangedEvent("test", cs, stateWithScripts())); + scriptService.compile(new Script(ScriptType.STORED, "test", "script", Collections.emptyMap()), randomFrom(scriptContexts)); assertEquals(1L, scriptService.stats().getCompilations()); } @@ -436,12 +426,8 @@ public void testDeleteScript() throws Exception { public void testGetStoredScript() throws Exception { buildScriptService(Settings.EMPTY); - ClusterState cs = ClusterState.builder(new ClusterName("_name")) - .metaData(MetaData.builder() - .putCustom(ScriptMetaData.TYPE, - new ScriptMetaData.Builder(null).storeScript("_id", - StoredScriptSource.parse("_lang", new BytesArray("{\"script\":\"abc\"}"), XContentType.JSON)).build())) - .build(); + ClusterState cs = stateWithScripts( + new Tuple<>("_id", StoredScriptSource.parse("_lang", new BytesArray("{\"script\":\"abc\"}"), XContentType.JSON))); assertEquals("abc", scriptService.getStoredScript(cs, new GetStoredScriptRequest("_id", "_lang")).getCode()); assertNull(scriptService.getStoredScript(cs, new GetStoredScriptRequest("_id2", "_lang"))); @@ -450,6 +436,18 @@ public void testGetStoredScript() throws Exception { assertNull(scriptService.getStoredScript(cs, new GetStoredScriptRequest("_id", "_lang"))); } + @SafeVarargs + private final ClusterState stateWithScripts(Tuple... scripts) { + ScriptMetaData.Builder builder = new ScriptMetaData.Builder(null); + for (Tuple script : scripts) { + builder.storeScript(script.v1(), script.v2()); + } + return ClusterState.builder(new ClusterName(getTestName())) + .metaData(MetaData.builder() + .putCustom(ScriptMetaData.TYPE, builder.build())) + .build(); + } + private void createFileScripts(String... langs) throws IOException { for (String lang : langs) { Path scriptPath = scriptsFilePath.resolve("file_script." + lang); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java index 16ba3ed6958b7..583406e226ac5 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java @@ -99,7 +99,7 @@ public void testParseAndValidate() { SearchContext context = mock(SearchContext.class); QueryShardContext qsc = new QueryShardContext(0, new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings), null, null, null, null, - null, xContentRegistry(), null, null, () -> now); + null, null, xContentRegistry(), null, null, () -> now); when(context.getQueryShardContext()).thenReturn(qsc); FormatDateTimeFormatter formatter = Joda.forPattern("dateOptionalTime"); DocValueFormat format = new DocValueFormat.DateTime(formatter, DateTimeZone.UTC); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java index b82e822f6b04c..348c77f82bcf0 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptContextRegistry; import org.elasticsearch.script.ScriptEngineRegistry; +import org.elasticsearch.script.ScriptMetrics; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptSettings; import org.elasticsearch.script.ScriptType; @@ -83,7 +84,7 @@ protected ScriptService mockScriptService() { ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); try { return new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, scriptContextRegistry, - scriptSettings); + scriptSettings, new ScriptMetrics()); } catch (IOException e) { throw new ElasticsearchException(e); } diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java index d9eb76310d241..257f895f8751d 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java @@ -36,6 +36,7 @@ import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptContextRegistry; import org.elasticsearch.script.ScriptEngineRegistry; +import org.elasticsearch.script.ScriptMetrics; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptSettings; import org.elasticsearch.script.ScriptType; @@ -161,12 +162,12 @@ protected QueryShardContext queryShardContextMock(final MappedFieldType[] fieldT ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); ScriptService scriptService; try { - scriptService = new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, scriptContextRegistry, - scriptSettings); + scriptService = new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, scriptContextRegistry, + scriptSettings, new ScriptMetrics()); } catch (IOException e) { throw new ElasticsearchException(e); } - return new QueryShardContext(0, idxSettings, null, null, null, null, scriptService, xContentRegistry(), + return new QueryShardContext(0, idxSettings, null, null, null, null, scriptService, null, xContentRegistry(), null, null, System::currentTimeMillis); } } diff --git a/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java b/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java index e33b201bf2252..f06ecbb87da2f 100644 --- a/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java @@ -265,7 +265,7 @@ public void testBuildSearchContextHighlight() throws IOException { Index index = new Index(randomAsciiOfLengthBetween(1, 10), "_na_"); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings); // shard context will only need indicesQueriesRegistry for building Query objects nested in highlighter - QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, null, null, null, null, null, xContentRegistry(), + QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, null, null, null, null, null, null, xContentRegistry(), null, null, System::currentTimeMillis) { @Override public MappedFieldType fieldMapper(String name) { diff --git a/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java b/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java index d0132cca7ad50..2257276ec856c 100644 --- a/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java +++ b/core/src/test/java/org/elasticsearch/search/internal/ShardSearchTransportRequestTests.java @@ -200,7 +200,7 @@ public void testSerialize50Request() throws IOException { IndexSettings indexSettings = new IndexSettings(indexMetadata.build(), Settings.EMPTY); final long nowInMillis = randomNonNegativeLong(); QueryShardContext context = new QueryShardContext( - 0, indexSettings, null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis); + 0, indexSettings, null, null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis); readRequest.rewrite(context); QueryBuilder queryBuilder = readRequest.filteringAliases(); assertEquals(queryBuilder, QueryBuilders.boolQuery() diff --git a/core/src/test/java/org/elasticsearch/search/rescore/QueryRescoreBuilderTests.java b/core/src/test/java/org/elasticsearch/search/rescore/QueryRescoreBuilderTests.java index 53277bd5b3c0b..543931a497339 100644 --- a/core/src/test/java/org/elasticsearch/search/rescore/QueryRescoreBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/search/rescore/QueryRescoreBuilderTests.java @@ -137,7 +137,7 @@ public void testBuildRescoreSearchContext() throws ElasticsearchParseException, .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAsciiOfLengthBetween(1, 10), indexSettings); // shard context will only need indicesQueriesRegistry for building Query objects nested in query rescorer - QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, null, null, null, null, null, xContentRegistry(), + QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, null, null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis) { @Override public MappedFieldType fieldMapper(String name) { diff --git a/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java b/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java index 17c60152aaefc..3daf84ad33ad0 100644 --- a/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java +++ b/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java @@ -55,6 +55,7 @@ import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptContextRegistry; import org.elasticsearch.script.ScriptEngineRegistry; +import org.elasticsearch.script.ScriptMetrics; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptServiceTests.TestEngineService; import org.elasticsearch.script.ScriptSettings; @@ -70,7 +71,6 @@ import java.io.IOException; import java.nio.file.Path; import java.util.Collections; -import java.util.Map; import static java.util.Collections.emptyList; import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode; @@ -94,8 +94,8 @@ public static void init() throws IOException { ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singletonList(new TestEngineService())); ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); - scriptService = new ScriptService(baseSettings, environment, - new ResourceWatcherService(baseSettings, null), scriptEngineRegistry, scriptContextRegistry, scriptSettings) { + scriptService = new ScriptService(baseSettings, environment, new ResourceWatcherService(baseSettings, null), scriptEngineRegistry, + scriptContextRegistry, scriptSettings, new ScriptMetrics()) { @Override public CompiledScript compile(Script script, ScriptContext scriptContext) { return new CompiledScript(ScriptType.INLINE, "mockName", "test", script); @@ -208,7 +208,7 @@ public void onCache(ShardId shardId, Accountable accountable) { } }); long nowInMillis = randomNonNegativeLong(); - return new QueryShardContext(0, idxSettings, bitsetFilterCache, ifds, null, null, scriptService, + return new QueryShardContext(0, idxSettings, bitsetFilterCache, ifds, null, null, scriptService, null, xContentRegistry(), null, null, () -> nowInMillis) { @Override public MappedFieldType fieldMapper(String name) { @@ -251,7 +251,6 @@ protected static QueryBuilder randomNestedFilter() { } } - @SuppressWarnings("unchecked") private T copy(T original) throws IOException { /* The cast below is required to make Java 9 happy. Java 8 infers the T in copyWriterable to be the same as AbstractSortTestCase's * T but Java 9 infers it to be SortBuilder. */ diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java index 9315a0fbd4b1f..899c02333ca45 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java @@ -33,7 +33,7 @@ import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; -import org.elasticsearch.script.ScriptEngineService; +import org.elasticsearch.script.TemplateService.Backend; import java.util.Arrays; import java.util.List; @@ -42,9 +42,8 @@ import static java.util.Collections.singletonList; public class MustachePlugin extends Plugin implements ScriptPlugin, ActionPlugin, SearchPlugin { - @Override - public ScriptEngineService getScriptEngineService(Settings settings) { + public Backend getTemplateBackend() { return new MustacheScriptEngineService(); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java index 2d39eb080e004..48b652d0fefc4 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java @@ -34,8 +34,8 @@ import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.GeneralScriptException; import org.elasticsearch.script.Script; -import org.elasticsearch.script.ScriptEngineService; import org.elasticsearch.script.SearchScript; +import org.elasticsearch.script.TemplateService; import org.elasticsearch.search.lookup.SearchLookup; import java.io.Reader; @@ -53,7 +53,7 @@ * process: First compile the string representing the template, the resulting * {@link Mustache} object can then be re-used for subsequent executions. */ -public final class MustacheScriptEngineService implements ScriptEngineService { +public final class MustacheScriptEngineService implements TemplateService.Backend { private static final Logger logger = ESLoggerFactory.getLogger(MustacheScriptEngineService.class); public static final String NAME = "mustache"; diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheTemplateService.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheTemplateService.java new file mode 100644 index 0000000000000..09da37cb8a2b8 --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheTemplateService.java @@ -0,0 +1,41 @@ +/* + * 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.mustache; + +import org.elasticsearch.common.settings.Settings; + +import java.io.Closeable; +import java.io.IOException; + +public class MustacheTemplateService implements Closeable { + private final Settings settings; + private final MustacheScriptEngineService engine; + + public MustacheTemplateService(Settings settings, MustacheScriptEngineService engine) { + this.settings = settings; + this.engine = engine; + } + + @Override + public void close() throws IOException { + engine.close(); + } + +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateAction.java new file mode 100644 index 0000000000000..de7e91024f18b --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateAction.java @@ -0,0 +1,42 @@ +/* + * 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.mustache; + +import org.elasticsearch.action.Action; +import org.elasticsearch.client.ElasticsearchClient; + +public class PutSearchTemplateAction extends Action { + public static final PutSearchTemplateAction INSTANCE = new PutSearchTemplateAction(); + public static final String NAME = "cluster:admin/template/put"; + + private PutSearchTemplateAction() { + super(NAME); + } + + @Override + public PutSearchTemplateResponse newResponse() { + return new PutSearchTemplateResponse(); + } + + @Override + public PutSearchTemplateRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new PutSearchTemplateRequestBuilder(client, this); + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequest.java new file mode 100644 index 0000000000000..9bd74346aca49 --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequest.java @@ -0,0 +1,108 @@ +/* + * 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.mustache; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.master.AcknowledgedRequest; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentType; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +public class PutSearchTemplateRequest extends AcknowledgedRequest { + private String id; + private BytesReference content; + private XContentType xContentType; + + public PutSearchTemplateRequest() { + super(); + } + + public PutSearchTemplateRequest(String id, BytesReference content, XContentType xContentType) { + super(); + this.id = id; + this.content = content; + this.xContentType = Objects.requireNonNull(xContentType); + } + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + + if (id == null || id.isEmpty()) { + validationException = addValidationError("must specify id for stored search template", validationException); + } else if (id.contains("#")) { + validationException = addValidationError("id cannot contain '#' for search template", validationException); + } + + if (content == null) { + validationException = addValidationError("must specify code for search template", validationException); + } + + return validationException; + } + + public String id() { + return id; + } + + public BytesReference content() { + return content; + } + + public XContentType xContentType() { + return xContentType; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + id = in.readOptionalString(); + content = in.readBytesReference(); + xContentType = XContentType.readFrom(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeOptionalString(id); + out.writeBytesReference(content); + xContentType.writeTo(out); + } + + @Override + public String toString() { + String source = "_na_"; + + try { + source = XContentHelper.convertToJson(content, false, xContentType); + } catch (Exception e) { + // ignore + } + + return "put search template {id [" + id + "], content [" + source + "]}"; + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequestBuilder.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequestBuilder.java new file mode 100644 index 0000000000000..e61264469ed15 --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequestBuilder.java @@ -0,0 +1,31 @@ +/* + * 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.mustache; + +import org.elasticsearch.action.support.master.AcknowledgedRequestBuilder; +import org.elasticsearch.client.ElasticsearchClient; + +public class PutSearchTemplateRequestBuilder + extends AcknowledgedRequestBuilder { + + public PutSearchTemplateRequestBuilder(ElasticsearchClient client, PutSearchTemplateAction action) { + super(client, action, new PutSearchTemplateRequest()); + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateResponse.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateResponse.java new file mode 100644 index 0000000000000..d7665ede89d2a --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateResponse.java @@ -0,0 +1,47 @@ +/* + * 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.mustache; + +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +public class PutSearchTemplateResponse extends AcknowledgedResponse { + PutSearchTemplateResponse() { + } + + public PutSearchTemplateResponse(boolean acknowledged) { + super(acknowledged); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + readAcknowledged(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + writeAcknowledged(out); + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java index d7b0406238278..7ae9ccab93ecd 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java @@ -36,7 +36,7 @@ import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.Script; -import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.TemplateService; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -46,21 +46,18 @@ import static org.elasticsearch.script.ScriptContext.Standard.SEARCH; public class TransportSearchTemplateAction extends HandledTransportAction { - - private static final String TEMPLATE_LANG = MustacheScriptEngineService.NAME; - - private final ScriptService scriptService; + private final TemplateService templateService; private final TransportSearchAction searchAction; private final NamedXContentRegistry xContentRegistry; @Inject public TransportSearchTemplateAction(Settings settings, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver resolver, - ScriptService scriptService, + TemplateService templateService, TransportSearchAction searchAction, NamedXContentRegistry xContentRegistry) { super(settings, SearchTemplateAction.NAME, threadPool, transportService, actionFilters, resolver, SearchTemplateRequest::new); - this.scriptService = scriptService; + this.templateService = templateService; this.searchAction = searchAction; this.xContentRegistry = xContentRegistry; } @@ -69,9 +66,8 @@ public TransportSearchTemplateAction(Settings settings, ThreadPool threadPool, T protected void doExecute(SearchTemplateRequest request, ActionListener listener) { final SearchTemplateResponse response = new SearchTemplateResponse(); try { - Script script = new Script(request.getScriptType(), TEMPLATE_LANG, request.getScript(), - request.getScriptParams() == null ? Collections.emptyMap() : request.getScriptParams()); - ExecutableScript executable = scriptService.executable(script, SEARCH); + ExecutableScript executable = templateService.executable(request.getScript(), request.getScriptType(), SEARCH, + request.getScriptParams() == null ? Collections.emptyMap() : request.getScriptParams()); BytesReference source = (BytesReference) executable.run(); response.setSource(source); diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/TemplateQueryBuilderTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/TemplateQueryBuilderTests.java index 3b70c5df626a5..f3ed3ed68f206 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/TemplateQueryBuilderTests.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/TemplateQueryBuilderTests.java @@ -75,7 +75,6 @@ protected Collection> getPlugins() { public static class CustomScriptPlugin extends MockScriptPlugin { @Override - @SuppressWarnings("unchecked") protected Map, Object>> pluginScripts() { Map, Object>> scripts = new HashMap<>(); diff --git a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/10_basic.yaml b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/10_basic.yaml index 644319d50ec3c..8f1c581c79c07 100644 --- a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/10_basic.yaml +++ b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/10_basic.yaml @@ -13,7 +13,7 @@ - match: { nodes.$master.modules.0.name: lang-mustache } --- -"Indexed template": +"Stored template": - do: put_template: diff --git a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java b/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java index 95810e089a275..2c6176b8f6fa5 100644 --- a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java +++ b/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java @@ -23,6 +23,7 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.script.ScriptContextRegistry; import org.elasticsearch.script.ScriptEngineRegistry; +import org.elasticsearch.script.ScriptMetrics; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptSettings; import org.elasticsearch.script.mustache.MustacheScriptEngineService; @@ -47,7 +48,7 @@ public void init() throws Exception { ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); ScriptService scriptService = new ScriptService(settings, new Environment(settings), null, - scriptEngineRegistry, scriptContextRegistry, scriptSettings); + scriptEngineRegistry, scriptContextRegistry, scriptSettings, new ScriptMetrics()); templateService = new InternalTemplateService(scriptService); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java index 2a2301069943a..d726a3fb6e6e2 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java @@ -20,6 +20,7 @@ package org.elasticsearch.test; import com.fasterxml.jackson.core.io.JsonStringEncoder; + import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; @@ -89,6 +90,7 @@ import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.script.ScriptModule; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.TemplateService; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.internal.SearchContext; import org.joda.time.DateTime; @@ -999,6 +1001,7 @@ private static class ServiceHolder implements Closeable { private final MapperService mapperService; private final BitsetFilterCache bitsetFilterCache; private final ScriptService scriptService; + private final TemplateService templateService; private final Client client; private final long nowInMillis = randomNonNegativeLong(); @@ -1030,6 +1033,7 @@ private static class ServiceHolder implements Closeable { AnalysisModule analysisModule = new AnalysisModule(new Environment(nodeSettings), emptyList()); IndexAnalyzers indexAnalyzers = analysisModule.getAnalysisRegistry().build(idxSettings); scriptService = scriptModule.getScriptService(); + templateService = scriptModule.getTemplateService(); similarityService = new SimilarityService(idxSettings, Collections.emptyMap()); MapperRegistry mapperRegistry = indicesModule.getMapperRegistry(); mapperService = new MapperService(idxSettings, indexAnalyzers, xContentRegistry, similarityService, mapperRegistry, @@ -1079,7 +1083,7 @@ public void close() throws IOException { QueryShardContext createShardContext() { return new QueryShardContext(0, idxSettings, bitsetFilterCache, indexFieldDataService, mapperService, similarityService, - scriptService, xContentRegistry, this.client, null, () -> nowInMillis); + scriptService, templateService, xContentRegistry, this.client, null, () -> nowInMillis); } ScriptModule createScriptModule(List scriptPlugins) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index 1ea96e6f54808..e8ebbbe3b9fac 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -1146,7 +1146,7 @@ public static ScriptModule newTestScriptModule() { .build(); Environment environment = new Environment(settings); MockScriptEngine scriptEngine = new MockScriptEngine(MockScriptEngine.NAME, Collections.singletonMap("1", script -> "1")); - return new ScriptModule(settings, environment, null, singletonList(scriptEngine), emptyList()); + return new ScriptModule(settings, environment, null, singletonList(scriptEngine), emptyList(), null); } /** Creates an IndicesModule for testing with the given mappers and metadata mappers. */ diff --git a/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java b/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java index 5f7e38b7ec1ce..d01244c2399b4 100644 --- a/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java +++ b/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java @@ -40,7 +40,7 @@ public class MockSearchServiceTests extends ESTestCase { public void testAssertNoInFlightContext() { final long nowInMillis = randomNonNegativeLong(); SearchContext s = new TestSearchContext(new QueryShardContext(0, - new IndexSettings(EMPTY_INDEX_METADATA, Settings.EMPTY), null, null, null, null, null, xContentRegistry(), + new IndexSettings(EMPTY_INDEX_METADATA, Settings.EMPTY), null, null, null, null, null, null, xContentRegistry(), null, null, () -> nowInMillis)) { @Override From 4867848bf436a3639c206c90965765da843e0e06 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 3 Mar 2017 14:07:24 -0500 Subject: [PATCH 02/37] I feel dirty --- .../elasticsearch/script/CachingCompiler.java | 83 ++----------- .../script/ScriptEngineRegistry.java | 5 +- .../elasticsearch/script/ScriptMetaData.java | 4 +- .../org/elasticsearch/script/ScriptModes.java | 2 +- .../elasticsearch/script/ScriptModule.java | 3 +- .../elasticsearch/script/ScriptService.java | 66 +++++++++-- .../elasticsearch/script/ScriptSettings.java | 24 ++-- .../elasticsearch/script/TemplateService.java | 18 +-- .../action/update/UpdateRequestTests.java | 2 +- .../elasticsearch/index/IndexModuleTests.java | 4 +- .../elasticsearch/script/FileScriptTests.java | 2 +- .../script/NativeScriptTests.java | 2 +- .../script/ScriptContextTests.java | 2 +- .../script/ScriptModesTests.java | 2 +- .../script/ScriptPermitsTests.java | 2 +- .../script/ScriptServiceTests.java | 2 +- .../script/ScriptSettingsTests.java | 7 +- .../script/TemplateServiceTests.java | 98 ++++++++++++++++ .../scripted/InternalScriptedMetricTests.java | 2 +- .../ScriptedMetricAggregatorTests.java | 2 +- .../search/sort/AbstractSortTestCase.java | 2 +- .../script/mustache/MustachePlugin.java | 20 +++- .../DeleteStoredSearchTemplateAction.java | 44 +++++++ .../DeleteStoredSearchTemplateRequest.java | 79 +++++++++++++ ...eteStoredSearchTemplateRequestBuilder.java | 33 ++++++ .../DeleteStoredSearchTemplateResponse.java | 48 ++++++++ .../GetStoredSearchTemplateAction.java} | 21 ++-- .../GetStoredSearchTemplateRequest.java | 77 +++++++++++++ ...GetStoredSearchTemplateRequestBuilder.java | 33 ++++++ .../GetStoredSearchTemplateResponse.java | 78 +++++++++++++ .../stored/PutStoredSearchTemplateAction.java | 43 +++++++ .../PutStoredSearchTemplateRequest.java} | 12 +- ...utStoredSearchTemplateRequestBuilder.java} | 12 +- .../PutStoredSearchTemplateResponse.java} | 8 +- ...RestDeleteStoredSearchTemplateAction.java} | 12 +- .../RestGetStoredSearchTemplateAction.java} | 43 ++++--- .../RestPutStoredSearchTemplateAction.java} | 12 +- ...sportDeleteStoredSearchTemplateAction.java | 88 ++++++++++++++ ...ransportGetStoredSearchTemplateAction.java | 79 +++++++++++++ ...ransportPutStoredSearchTemplateAction.java | 109 ++++++++++++++++++ .../mustache/TemplateQueryBuilderTests.java | 38 +----- .../ingest/AbstractScriptTestCase.java | 2 +- 42 files changed, 999 insertions(+), 226 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequestBuilder.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateResponse.java rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/{PutSearchTemplateAction.java => stored/GetStoredSearchTemplateAction.java} (57%) create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequestBuilder.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateResponse.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateAction.java rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/{PutSearchTemplateRequest.java => stored/PutStoredSearchTemplateRequest.java} (88%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/{PutSearchTemplateRequestBuilder.java => stored/PutStoredSearchTemplateRequestBuilder.java} (66%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/{PutSearchTemplateResponse.java => stored/PutStoredSearchTemplateResponse.java} (85%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/{RestDeleteSearchTemplateAction.java => stored/RestDeleteStoredSearchTemplateAction.java} (71%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/{RestGetSearchTemplateAction.java => stored/RestGetStoredSearchTemplateAction.java} (56%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/{RestPutSearchTemplateAction.java => stored/RestPutStoredSearchTemplateAction.java} (74%) create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredSearchTemplateAction.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredSearchTemplateAction.java create mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 7f0c7f099c926..35c85b9b4329e 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -24,18 +24,9 @@ import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ResourceNotFoundException; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.cluster.storedscripts.DeleteStoredScriptRequest; -import org.elasticsearch.action.admin.cluster.storedscripts.DeleteStoredScriptResponse; -import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptRequest; -import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptRequest; -import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptResponse; -import org.elasticsearch.cluster.AckedClusterStateUpdateTask; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateListener; -import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.cache.Cache; import org.elasticsearch.common.cache.CacheBuilder; import org.elasticsearch.common.cache.RemovalListener; @@ -80,8 +71,6 @@ public abstract class CachingCompiler implements ClusterStateListener private final Path scriptsDirectory; - private final int maxScriptSizeInBytes; - private final ScriptMetrics scriptMetrics; private volatile ClusterState clusterState; @@ -118,7 +107,6 @@ public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environ fileWatcher.init(); } - maxScriptSizeInBytes = ScriptService.SCRIPT_MAX_SIZE_IN_BYTES.get(settings); this.scriptMetrics = scriptMetrics; } @@ -128,7 +116,6 @@ public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environ protected abstract CacheKeyT cacheKeyForFile(String baseName, String extension); protected abstract CacheKeyT cacheKeyFromClusterState(StoredScriptSource scriptMetadata); protected abstract StoredScriptSource lookupStoredScript(ClusterState clusterState, CacheKeyT cacheKey); - protected abstract void checkPutSupported(StoredScriptSource source); /** * Are any script contexts enabled for the given {@code cacheKey} and {@code scriptType}? Used to reject compilation if all script * contexts are disabled and produce a nice error message earlier rather than later. @@ -209,81 +196,25 @@ private CacheKeyT getScriptFromClusterState(CacheKeyT cacheKey) { return cacheKeyFromClusterState(source); } - public final void putStoredScript(ClusterService clusterService, PutStoredScriptRequest request, - ActionListener listener) { - if (request.content().length() > maxScriptSizeInBytes) { - throw new IllegalArgumentException("exceeded max allowed stored script size in bytes [" + maxScriptSizeInBytes - + "] with size [" + request.content().length() + "] for script [" + request.id() + "]"); - } - - StoredScriptSource source = StoredScriptSource.parse(request.lang(), request.content(), request.xContentType()); - checkPutSupported(source); + /** + * Check that a script compiles before attempting to store it. + */ + public final void checkCompileBeforeStore(StoredScriptSource source) { CacheKeyT cacheKey = cacheKeyFromClusterState(source); - try { if (areAnyScriptContextsEnabled(cacheKey, ScriptType.STORED)) { Object compiled = compile(ScriptType.STORED, cacheKey); if (compiled == null) { - throw new IllegalArgumentException("failed to parse/compile stored script [" + request.id() + "]" + - (source.getCode() == null ? "" : " using code [" + source.getCode() + "]")); + throw new IllegalArgumentException("failed to parse/compile"); } } else { - throw new IllegalArgumentException( - "cannot put stored script [" + request.id() + "], stored scripts cannot be run under any context"); + throw new IllegalArgumentException("cannot be run under any context"); } } catch (ScriptException good) { throw good; } catch (Exception exception) { - throw new IllegalArgumentException("failed to parse/compile stored script [" + request.id() + "]", exception); - } - - clusterService.submitStateUpdateTask("put-script-" + request.id(), - new AckedClusterStateUpdateTask(request, listener) { - - @Override - protected PutStoredScriptResponse newResponse(boolean acknowledged) { - return new PutStoredScriptResponse(acknowledged); - } - - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); - smd = ScriptMetaData.putStoredScript(smd, request.id(), source); - MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); - - return ClusterState.builder(currentState).metaData(mdb).build(); - } - }); - } - - public final void deleteStoredScript(ClusterService clusterService, DeleteStoredScriptRequest request, - ActionListener listener) { - clusterService.submitStateUpdateTask("delete-script-" + request.id(), - new AckedClusterStateUpdateTask(request, listener) { - @Override - protected DeleteStoredScriptResponse newResponse(boolean acknowledged) { - return new DeleteStoredScriptResponse(acknowledged); - } - - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); - smd = ScriptMetaData.deleteStoredScript(smd, request.id(), request.lang()); - MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); - - return ClusterState.builder(currentState).metaData(mdb).build(); - } - }); - } - - public final StoredScriptSource getStoredScript(ClusterState state, GetStoredScriptRequest request) { - ScriptMetaData scriptMetadata = state.metaData().custom(ScriptMetaData.TYPE); - - if (scriptMetadata != null) { - return scriptMetadata.getStoredScript(request.id(), request.lang()); - } else { - return null; + throw new IllegalArgumentException("failed to parse/compile", exception); } } diff --git a/core/src/main/java/org/elasticsearch/script/ScriptEngineRegistry.java b/core/src/main/java/org/elasticsearch/script/ScriptEngineRegistry.java index f65d694aa3181..f4209523b63b2 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptEngineRegistry.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptEngineRegistry.java @@ -19,12 +19,11 @@ package org.elasticsearch.script; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Objects; -import org.elasticsearch.common.Strings; public class ScriptEngineRegistry { @@ -59,7 +58,7 @@ public ScriptEngineRegistry(Iterable registrations) { this.defaultInlineScriptEnableds = Collections.unmodifiableMap(inlineScriptEnableds); } - Iterable> getRegisteredScriptEngineServices() { + Collection> getRegisteredScriptEngineServices() { return registeredScriptEngineServices.keySet(); } diff --git a/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java b/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java index 785c8f30782bd..62137e46f532a 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptMetaData.java @@ -182,7 +182,7 @@ public void writeTo(StreamOutput out) throws IOException { * Convenience method to build and return a new * {@link ScriptMetaData} adding the specified stored script. */ - static ScriptMetaData putStoredScript(ScriptMetaData previous, String id, StoredScriptSource source) { + public static ScriptMetaData putStoredScript(ScriptMetaData previous, String id, StoredScriptSource source) { Builder builder = new Builder(previous); builder.storeScript(id, source); @@ -193,7 +193,7 @@ static ScriptMetaData putStoredScript(ScriptMetaData previous, String id, Stored * Convenience method to build and return a new * {@link ScriptMetaData} deleting the specified stored script. */ - static ScriptMetaData deleteStoredScript(ScriptMetaData previous, String id, String lang) { + public static ScriptMetaData deleteStoredScript(ScriptMetaData previous, String id, String lang) { Builder builder = new ScriptMetaData.Builder(previous); builder.deleteScript(id, lang); diff --git a/core/src/main/java/org/elasticsearch/script/ScriptModes.java b/core/src/main/java/org/elasticsearch/script/ScriptModes.java index 439a09848d4cb..ec97cbe031ef9 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptModes.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptModes.java @@ -47,7 +47,7 @@ public class ScriptModes { } /** - * Returns the script mode for a script of a certain written in a certain language, + * Returns the script mode for a script or template of a certain written in a certain language, * of a certain type and executing as part of a specific operation/api. * * @param lang the language that the script is written in diff --git a/core/src/main/java/org/elasticsearch/script/ScriptModule.java b/core/src/main/java/org/elasticsearch/script/ScriptModule.java index 59d13a9ba04a0..4b07b80dd7368 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptModule.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptModule.java @@ -84,7 +84,8 @@ public ScriptModule(Settings settings, Environment environment, ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(customScriptContexts); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(scriptEngineServices); ScriptMetrics scriptMetrics = new ScriptMetrics(); - scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + // Note that if templateBackend is null this won't register any settings for it + scriptSettings = new ScriptSettings(scriptEngineRegistry, templateBackend, scriptContextRegistry); try { scriptService = new ScriptService(settings, environment, resourceWatcherService, scriptEngineRegistry, scriptContextRegistry, scriptSettings, scriptMetrics); diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index 80afb80058e02..c7d8246efaac4 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -26,9 +26,11 @@ import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptRequest; import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptRequest; import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptResponse; +import org.elasticsearch.cluster.AckedClusterStateUpdateTask; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.component.AbstractComponent; @@ -73,6 +75,7 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust private final ScriptMetrics scriptMetrics; private final ScriptPermits scriptPermits; private final CachingCompiler compiler; + private final int maxScriptSizeInBytes; public ScriptService(Settings settings, Environment env, ResourceWatcherService resourceWatcherService, ScriptEngineRegistry scriptEngineRegistry, ScriptContextRegistry scriptContextRegistry, ScriptSettings scriptSettings, @@ -98,6 +101,7 @@ public ScriptService(Settings settings, Environment env, ResourceWatcherService this.scriptEnginesByLang = unmodifiableMap(enginesByLangBuilder); this.scriptEnginesByExt = unmodifiableMap(enginesByExtBuilder); + maxScriptSizeInBytes = ScriptService.SCRIPT_MAX_SIZE_IN_BYTES.get(settings); this.scriptMetrics = scriptMetrics; this.scriptPermits = new ScriptPermits(settings, scriptSettings, scriptContextRegistry); this.compiler = new CachingCompiler(settings, scriptSettings, env, resourceWatcherService, scriptMetrics) { @@ -120,13 +124,6 @@ protected CacheKey cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { return new CacheKey(scriptMetadata.getLang(), scriptMetadata.getCode(), scriptMetadata.getOptions()); } - @Override - protected void checkPutSupported(StoredScriptSource source) { - if (false == isLangSupported(source.getLang())) { - throw new IllegalArgumentException("unable to put stored script with unsupported lang [" + source.getLang() + "]"); - } - } - @Override protected StoredScriptSource lookupStoredScript(ClusterState clusterState, CacheKey cacheKey) { ScriptMetaData scriptMetadata = clusterState.metaData().custom(ScriptMetaData.TYPE); @@ -291,12 +288,46 @@ public void clusterChanged(ClusterChangedEvent event) { } public final StoredScriptSource getStoredScript(ClusterState state, GetStoredScriptRequest request) { - return compiler.getStoredScript(state, request); + ScriptMetaData scriptMetadata = state.metaData().custom(ScriptMetaData.TYPE); + + if (scriptMetadata != null) { + return scriptMetadata.getStoredScript(request.id(), request.lang()); + } else { + return null; + } } public void putStoredScript(ClusterService clusterService, PutStoredScriptRequest request, ActionListener listener) { - compiler.putStoredScript(clusterService, request, listener); + if (request.content().length() > maxScriptSizeInBytes) { + throw new IllegalArgumentException("exceeded max allowed stored script size in bytes [" + maxScriptSizeInBytes + + "] with size [" + request.content().length() + "] for script [" + request.id() + "]"); + } + + StoredScriptSource source = StoredScriptSource.parse(request.lang(), request.content(), request.xContentType()); + try { + compiler.checkCompileBeforeStore(source); + } catch (IllegalArgumentException | ScriptException e) { + throw new IllegalArgumentException("failed to parse/compile stored script [" + request.id() + "]" + + (source.getCode() == null ? "" : " using code [" + source.getCode() + "]"), e); + } + clusterService.submitStateUpdateTask("put-script-" + request.id(), + new AckedClusterStateUpdateTask(request, listener) { + + @Override + protected PutStoredScriptResponse newResponse(boolean acknowledged) { + return new PutStoredScriptResponse(acknowledged); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); + smd = ScriptMetaData.putStoredScript(smd, request.id(), source); + MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); + + return ClusterState.builder(currentState).metaData(mdb).build(); + } + }); } public void deleteStoredScript(ClusterService clusterService, DeleteStoredScriptRequest request, @@ -305,7 +336,22 @@ public void deleteStoredScript(ClusterService clusterService, DeleteStoredScript throw new IllegalArgumentException("unable to delete stored script with unsupported lang [" + request.lang() + "]"); } - compiler.deleteStoredScript(clusterService, request, listener); + clusterService.submitStateUpdateTask("delete-script-" + request.id(), + new AckedClusterStateUpdateTask(request, listener) { + @Override + protected DeleteStoredScriptResponse newResponse(boolean acknowledged) { + return new DeleteStoredScriptResponse(acknowledged); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); + smd = ScriptMetaData.deleteStoredScript(smd, request.id(), request.lang()); + MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); + + return ClusterState.builder(currentState).metaData(mdb).build(); + } + }); } private static final class CacheKey { diff --git a/core/src/main/java/org/elasticsearch/script/ScriptSettings.java b/core/src/main/java/org/elasticsearch/script/ScriptSettings.java index 5ee18862b3e99..f1c5ac9aa8b44 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptSettings.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptSettings.java @@ -28,6 +28,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.BiConsumer; import java.util.function.Function; public class ScriptSettings { @@ -48,12 +49,13 @@ public class ScriptSettings { private final Map> scriptContextSettingMap; private final List> scriptLanguageSettings; - public ScriptSettings(ScriptEngineRegistry scriptEngineRegistry, ScriptContextRegistry scriptContextRegistry) { + public ScriptSettings(ScriptEngineRegistry scriptEngineRegistry, TemplateService.Backend templateBackend, + ScriptContextRegistry scriptContextRegistry) { Map> scriptContextSettingMap = contextSettings(scriptContextRegistry); this.scriptContextSettingMap = Collections.unmodifiableMap(scriptContextSettingMap); List> scriptLanguageSettings = languageSettings(SCRIPT_TYPE_SETTING_MAP, scriptContextSettingMap, - scriptEngineRegistry, scriptContextRegistry); + scriptEngineRegistry, templateBackend, scriptContextRegistry); this.scriptLanguageSettings = Collections.unmodifiableList(scriptLanguageSettings); } @@ -69,18 +71,13 @@ private static Map> contextSettings(ScriptContex private static List> languageSettings(Map> scriptTypeSettingMap, Map> scriptContextSettingMap, ScriptEngineRegistry scriptEngineRegistry, + TemplateService.Backend templateBackend, ScriptContextRegistry scriptContextRegistry) { final List> scriptModeSettings = new ArrayList<>(); - for (final Class scriptEngineService : scriptEngineRegistry.getRegisteredScriptEngineServices()) { - if (scriptEngineService == NativeScriptEngineService.class) { - // native scripts are always enabled, and their settings can not be changed - continue; - } - final String language = scriptEngineRegistry.getLanguage(scriptEngineService); + BiConsumer populate = (language, defaultNonFileScriptMode) -> { for (final ScriptType scriptType : ScriptType.values()) { // Top level, like "script.engine.groovy.inline" - final boolean defaultNonFileScriptMode = scriptEngineRegistry.getDefaultInlineScriptEnableds().get(language); boolean defaultLangAndType = defaultNonFileScriptMode; // Files are treated differently because they are never default-deny if (ScriptType.FILE == scriptType) { @@ -142,6 +139,15 @@ private static List> languageSettings(Map c : scriptEngineRegistry.getRegisteredScriptEngineServices()) { + if (c != NativeScriptEngineService.class) { + String language = scriptEngineRegistry.getLanguage(c); + populate.accept(language, scriptEngineRegistry.getDefaultInlineScriptEnableds().get(language)); + } + } + if (templateBackend != null) { + populate.accept(templateBackend.getType(), true); // Templates are enabled by default } return scriptModeSettings; } diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index 5dfcac527b448..6e0dc44d2dfd9 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -65,13 +65,6 @@ protected String cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { return scriptMetadata.getCode(); } - @Override - protected void checkPutSupported(StoredScriptSource source) { - if (false == backend.getType().equals(source.getLang())) { - throw new IllegalArgumentException("only mustache scripts are supported here but got [" + source.getLang() + "]"); - } - } - @Override protected StoredScriptSource lookupStoredScript(ClusterState clusterState, String cacheKey) { ScriptMetaData scriptMetadata = clusterState.metaData().custom(ScriptMetaData.TYPE); @@ -138,4 +131,15 @@ public ExecutableScript executable(String idOrCode, ScriptType scriptType, Scrip Map scriptParams) { return backend.executable(compiler.getScript(idOrCode, scriptType, scriptContext), scriptParams); } + + /** + * The language name that templates have when stored in {@link ScriptMetaData}. + */ + public String getTemplateLanguage() { + return backend.getType(); + } + + public void checkCompileBeforeStore(StoredScriptSource source) { + compiler.checkCompileBeforeStore(source); + } } diff --git a/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java b/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java index a731ebee615fa..aef0b72a14329 100644 --- a/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java @@ -285,7 +285,7 @@ public void testNowInScript() throws IOException { ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singletonList(new MockScriptEngine("mock", scripts))); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); ScriptService scriptService = new ScriptService(baseSettings, environment, new ResourceWatcherService(baseSettings, null), scriptEngineRegistry, scriptContextRegistry, scriptSettings, new ScriptMetrics()); Settings settings = settings(Version.CURRENT).build(); diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 42d1d6ab3662a..6f62984ed882b 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -34,7 +34,6 @@ import org.apache.lucene.util.SetOnce.AlreadySetException; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; @@ -111,7 +110,6 @@ public void addPendingDelete(ShardId shardId, IndexSettings indexSettings) { private CircuitBreakerService circuitBreakerService; private BigArrays bigArrays; private ScriptService scriptService; - private ClusterService clusterService; @Override public void setUp() throws Exception { @@ -127,7 +125,7 @@ public void setUp() throws Exception { bigArrays = new BigArrays(settings, circuitBreakerService); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(emptyList()); ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); scriptService = new ScriptService(settings, environment, new ResourceWatcherService(settings, threadPool), scriptEngineRegistry, scriptContextRegistry, scriptSettings, new ScriptMetrics()); nodeEnvironment = new NodeEnvironment(settings, environment); diff --git a/core/src/test/java/org/elasticsearch/script/FileScriptTests.java b/core/src/test/java/org/elasticsearch/script/FileScriptTests.java index 57d6d6b8f192b..94728002f38fa 100644 --- a/core/src/test/java/org/elasticsearch/script/FileScriptTests.java +++ b/core/src/test/java/org/elasticsearch/script/FileScriptTests.java @@ -46,7 +46,7 @@ ScriptService makeScriptService(Settings settings) throws Exception { MockScriptEngine scriptEngine = new MockScriptEngine(MockScriptEngine.NAME, Collections.singletonMap(scriptSource, script -> "1")); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singleton(scriptEngine)); ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); return new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, scriptContextRegistry, scriptSettings, new ScriptMetrics()); } diff --git a/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java b/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java index 64f5378ee07a7..5c9622955eb9d 100644 --- a/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java +++ b/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java @@ -75,7 +75,7 @@ public void testFineGrainedSettingsDontAffectNativeScripts() throws IOException ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singleton(new NativeScriptEngineService(settings, nativeScriptFactoryMap))); ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(new ArrayList<>()); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); ScriptService scriptService = new ScriptService(settings, environment, resourceWatcherService, scriptEngineRegistry, scriptContextRegistry, scriptSettings, new ScriptMetrics()); diff --git a/core/src/test/java/org/elasticsearch/script/ScriptContextTests.java b/core/src/test/java/org/elasticsearch/script/ScriptContextTests.java index 855d35ea7d84b..94450d748562e 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptContextTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptContextTests.java @@ -53,7 +53,7 @@ ScriptService makeScriptService() throws Exception { new ScriptContext.Plugin(PLUGIN_NAME, "custom_exp_disabled_op"), new ScriptContext.Plugin(PLUGIN_NAME, "custom_globally_disabled_op")); ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(customContexts); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); // NOCOMMIT remove null and test ScriptService scriptService = new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, scriptContextRegistry, scriptSettings, new ScriptMetrics()); diff --git a/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java b/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java index f6a02ae920630..ce0f1b8b66d41 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java @@ -68,7 +68,7 @@ public void setupScriptEngines() { new NativeScriptEngineService(Settings.EMPTY, Collections.emptyMap()), new CustomScriptEngineService())); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(scriptEngines.values()); - scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); // NOCOMMIT remove null and test checkedSettings = new HashSet<>(); assertAllSettingsWereChecked = true; assertScriptModesNonNull = true; diff --git a/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java b/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java index 38b9e29121d93..601ef6f8de5d7 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java @@ -28,7 +28,7 @@ public class ScriptPermitsTests extends ESTestCase { public void testCompilationCircuitBreaking() throws Exception { ScriptContextRegistry contextRegistry = new ScriptContextRegistry(emptyList()); - ScriptSettings scriptSettings = new ScriptSettings(new ScriptEngineRegistry(emptyList()), contextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(new ScriptEngineRegistry(emptyList()), null, contextRegistry); ScriptPermits permits = new ScriptPermits(Settings.EMPTY, scriptSettings, contextRegistry); permits.setMaxCompilationsPerMinute(1); permits.checkCompilationLimit(); // should pass diff --git a/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java b/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java index 8634bb9e323f4..c95442171d7b0 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptServiceTests.java @@ -109,7 +109,7 @@ public void setup() throws IOException { scriptEngineRegistry = new ScriptEngineRegistry(Arrays.asList(scriptEngineService, dangerousScriptEngineService, defaultScriptServiceEngine)); scriptContextRegistry = new ScriptContextRegistry(contexts.values()); - scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); scriptContexts = scriptContextRegistry.scriptContexts().toArray(new ScriptContext[scriptContextRegistry.scriptContexts().size()]); logger.info("--> setup script service"); scriptsFilePath = genericConfigFolder.resolve("scripts"); diff --git a/core/src/test/java/org/elasticsearch/script/ScriptSettingsTests.java b/core/src/test/java/org/elasticsearch/script/ScriptSettingsTests.java index abda0376a2cfa..510b35e920fb6 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptSettingsTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptSettingsTests.java @@ -29,8 +29,9 @@ import java.util.Iterator; import java.util.Map; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class ScriptSettingsTests extends ESTestCase { @@ -38,7 +39,9 @@ public void testSettingsAreProperlyPropogated() { ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singletonList(new CustomScriptEngineService())); ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + TemplateService.Backend templateBackend = mock(TemplateService.Backend.class); + when(templateBackend.getType()).thenReturn("test_template_backend"); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, templateBackend, scriptContextRegistry); boolean enabled = randomBoolean(); Settings s = Settings.builder().put("script.inline", enabled).build(); for (Iterator> iter = scriptSettings.getScriptLanguageSettings().iterator(); iter.hasNext();) { diff --git a/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java b/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java new file mode 100644 index 0000000000000..a0312f766c778 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java @@ -0,0 +1,98 @@ +/* + * 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.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.watcher.ResourceWatcherService; +import org.junit.Before; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; + +import static java.util.Collections.emptyList; +import static java.util.Collections.emptyMap; +import static java.util.Collections.singletonMap; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TemplateServiceTests extends ESTestCase { + private ResourceWatcherService resourceWatcherService; + private ScriptContextRegistry scriptContextRegistry; + private ScriptSettings scriptSettings; + private ScriptContext[] scriptContexts; + private Path scriptsFilePath; + private Settings settings; + private TemplateService.Backend backend; + private TemplateService templateService; + + private static final Map DEFAULT_SCRIPT_ENABLED = new HashMap<>(); + static { + DEFAULT_SCRIPT_ENABLED.put(ScriptType.FILE, true); + DEFAULT_SCRIPT_ENABLED.put(ScriptType.STORED, false); + DEFAULT_SCRIPT_ENABLED.put(ScriptType.INLINE, false); + } + + @Before + public void setup() throws IOException { + Path genericConfigFolder = createTempDir(); + settings = Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) + .put(Environment.PATH_CONF_SETTING.getKey(), genericConfigFolder) + .put(ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE.getKey(), 10000) + .build(); + resourceWatcherService = new ResourceWatcherService(settings, null); + //randomly register custom script contexts + int randomInt = randomIntBetween(0, 3); + //prevent duplicates using map + Map contexts = new HashMap<>(); + for (int i = 0; i < randomInt; i++) { + String plugin; + do { + plugin = randomAsciiOfLength(randomIntBetween(1, 10)); + } while (ScriptContextRegistry.RESERVED_SCRIPT_CONTEXTS.contains(plugin)); + String operation; + do { + operation = randomAsciiOfLength(randomIntBetween(1, 30)); + } while (ScriptContextRegistry.RESERVED_SCRIPT_CONTEXTS.contains(operation)); + String context = plugin + "_" + operation; + contexts.put(context, new ScriptContext.Plugin(plugin, operation)); + } + scriptContextRegistry = new ScriptContextRegistry(contexts.values()); + backend = mock(TemplateService.Backend.class); + when(backend.getType()).thenReturn("test_template_backend"); + scriptSettings = new ScriptSettings(new ScriptEngineRegistry(emptyList()), backend, scriptContextRegistry); + scriptContexts = scriptContextRegistry.scriptContexts().toArray(new ScriptContext[scriptContextRegistry.scriptContexts().size()]); + logger.info("--> setup script service"); + scriptsFilePath = genericConfigFolder.resolve("scripts"); + Files.createDirectories(scriptsFilePath); + Environment environment = new Environment(settings); + templateService = new TemplateService(settings, environment, resourceWatcherService, backend, scriptContextRegistry, + scriptSettings, new ScriptMetrics()); + } + + public void testFoo() { + // NOCOMMIT something here + } +} diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java index 348c77f82bcf0..9daf89bbbde4b 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetricTests.java @@ -81,7 +81,7 @@ protected ScriptService mockScriptService() { })); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singletonList(scriptEngine)); ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); try { return new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, scriptContextRegistry, scriptSettings, new ScriptMetrics()); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java index 257f895f8751d..6a8c18c69ff63 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java @@ -159,7 +159,7 @@ protected QueryShardContext queryShardContextMock(final MappedFieldType[] fieldT MockScriptEngine scriptEngine = new MockScriptEngine(MockScriptEngine.NAME, SCRIPTS); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singletonList(scriptEngine)); ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); ScriptService scriptService; try { scriptService = new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, scriptContextRegistry, diff --git a/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java b/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java index 3daf84ad33ad0..831a7a41f6bdb 100644 --- a/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java +++ b/core/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java @@ -93,7 +93,7 @@ public static void init() throws IOException { Environment environment = new Environment(baseSettings); ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singletonList(new TestEngineService())); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); scriptService = new ScriptService(baseSettings, environment, new ResourceWatcherService(baseSettings, null), scriptEngineRegistry, scriptContextRegistry, scriptSettings, new ScriptMetrics()) { @Override diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java index 899c02333ca45..0ee2943a83cfb 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java @@ -34,6 +34,13 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; import org.elasticsearch.script.TemplateService.Backend; +import org.elasticsearch.script.mustache.stored.DeleteStoredSearchTemplateAction; +import org.elasticsearch.script.mustache.stored.GetStoredSearchTemplateAction; +import org.elasticsearch.script.mustache.stored.RestDeleteStoredSearchTemplateAction; +import org.elasticsearch.script.mustache.stored.RestGetStoredSearchTemplateAction; +import org.elasticsearch.script.mustache.stored.RestPutStoredSearchTemplateAction; +import org.elasticsearch.script.mustache.stored.TransportDeleteStoredSearchTemplateAction; +import org.elasticsearch.script.mustache.stored.TransportGetStoredSearchTemplateAction; import java.util.Arrays; import java.util.List; @@ -49,8 +56,11 @@ public Backend getTemplateBackend() { @Override public List> getActions() { - return Arrays.asList(new ActionHandler<>(SearchTemplateAction.INSTANCE, TransportSearchTemplateAction.class), - new ActionHandler<>(MultiSearchTemplateAction.INSTANCE, TransportMultiSearchTemplateAction.class)); + return Arrays.asList( + new ActionHandler<>(SearchTemplateAction.INSTANCE, TransportSearchTemplateAction.class), + new ActionHandler<>(MultiSearchTemplateAction.INSTANCE, TransportMultiSearchTemplateAction.class), + new ActionHandler<>(DeleteStoredSearchTemplateAction.INSTANCE, TransportDeleteStoredSearchTemplateAction.class), + new ActionHandler<>(GetStoredSearchTemplateAction.INSTANCE, TransportGetStoredSearchTemplateAction.class)); } @Override @@ -65,9 +75,9 @@ public List getRestHandlers(Settings settings, RestController restC return Arrays.asList( new RestSearchTemplateAction(settings, restController), new RestMultiSearchTemplateAction(settings, restController), - new RestGetSearchTemplateAction(settings, restController), - new RestPutSearchTemplateAction(settings, restController), - new RestDeleteSearchTemplateAction(settings, restController), + new RestGetStoredSearchTemplateAction(settings, restController), + new RestPutStoredSearchTemplateAction(settings, restController), + new RestDeleteStoredSearchTemplateAction(settings, restController), new RestRenderSearchTemplateAction(settings, restController)); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java new file mode 100644 index 0000000000000..1e511d2ed0009 --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java @@ -0,0 +1,44 @@ +/* + * 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.mustache.stored; + +import org.elasticsearch.action.Action; +import org.elasticsearch.client.ElasticsearchClient; + +public class DeleteStoredSearchTemplateAction + extends Action { + + public static final DeleteStoredSearchTemplateAction INSTANCE = new DeleteStoredSearchTemplateAction(); + public static final String NAME = "cluster:admin/script/delete"; + + private DeleteStoredSearchTemplateAction() { + super(NAME); + } + + @Override + public DeleteStoredSearchTemplateResponse newResponse() { + return new DeleteStoredSearchTemplateResponse(); + } + + @Override + public DeleteStoredSearchTemplateRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new DeleteStoredSearchTemplateRequestBuilder(client, this); + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java new file mode 100644 index 0000000000000..ed3958f15291d --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java @@ -0,0 +1,79 @@ +/* + * 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.mustache.stored; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.master.AcknowledgedRequest; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +public class DeleteStoredSearchTemplateRequest extends AcknowledgedRequest { + private String id; + + DeleteStoredSearchTemplateRequest() { + super(); + } + + public DeleteStoredSearchTemplateRequest(String id) { + super(); + + this.id = id; + } + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + + if (id == null || id.isEmpty()) { + validationException = addValidationError("must specify id for stored search template", validationException); + } else if (id.contains("#")) { + validationException = addValidationError("id cannot contain '#' for stored search template", validationException); + } + + return validationException; + } + + public String id() { + return id; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + + id = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + + out.writeString(id); + } + + @Override + public String toString() { + return "delete stored script {id [" + id + "]}"; + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequestBuilder.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequestBuilder.java new file mode 100644 index 0000000000000..b738f5871eb4e --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequestBuilder.java @@ -0,0 +1,33 @@ +/* + * 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.mustache.stored; + +import org.elasticsearch.action.support.master.AcknowledgedRequestBuilder; +import org.elasticsearch.client.ElasticsearchClient; + +public class DeleteStoredSearchTemplateRequestBuilder extends AcknowledgedRequestBuilder< + DeleteStoredSearchTemplateRequest, + DeleteStoredSearchTemplateResponse, + DeleteStoredSearchTemplateRequestBuilder> { + + public DeleteStoredSearchTemplateRequestBuilder(ElasticsearchClient client, DeleteStoredSearchTemplateAction action) { + super(client, action, new DeleteStoredSearchTemplateRequest()); + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateResponse.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateResponse.java new file mode 100644 index 0000000000000..823b86c387775 --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateResponse.java @@ -0,0 +1,48 @@ +/* + * 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.mustache.stored; + +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +public class DeleteStoredSearchTemplateResponse extends AcknowledgedResponse { + + DeleteStoredSearchTemplateResponse() { + } + + public DeleteStoredSearchTemplateResponse(boolean acknowledged) { + super(acknowledged); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + readAcknowledged(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + writeAcknowledged(out); + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateAction.java similarity index 57% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateAction.java index de7e91024f18b..3fbfe0f138379 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateAction.java @@ -17,26 +17,29 @@ * under the License. */ -package org.elasticsearch.script.mustache; +package org.elasticsearch.script.mustache.stored; import org.elasticsearch.action.Action; import org.elasticsearch.client.ElasticsearchClient; -public class PutSearchTemplateAction extends Action { - public static final PutSearchTemplateAction INSTANCE = new PutSearchTemplateAction(); - public static final String NAME = "cluster:admin/template/put"; +public class GetStoredSearchTemplateAction extends Action { - private PutSearchTemplateAction() { + public static final GetStoredSearchTemplateAction INSTANCE = new GetStoredSearchTemplateAction(); + public static final String NAME = "cluster:admin/script/get"; + + private GetStoredSearchTemplateAction() { super(NAME); } @Override - public PutSearchTemplateResponse newResponse() { - return new PutSearchTemplateResponse(); + public GetStoredSearchTemplateResponse newResponse() { + return new GetStoredSearchTemplateResponse(); } @Override - public PutSearchTemplateRequestBuilder newRequestBuilder(ElasticsearchClient client) { - return new PutSearchTemplateRequestBuilder(client, this); + public GetStoredSearchTemplateRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new GetStoredSearchTemplateRequestBuilder(client, this); } + } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java new file mode 100644 index 0000000000000..3d2ad61fba5bf --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java @@ -0,0 +1,77 @@ +/* + * 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.mustache.stored; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.master.MasterNodeReadRequest; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +public class GetStoredSearchTemplateRequest extends MasterNodeReadRequest { + protected String id; + + GetStoredSearchTemplateRequest() { + super(); + } + + public GetStoredSearchTemplateRequest(String id) { + super(); + + this.id = id; + } + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + + if (id == null || id.isEmpty()) { + validationException = addValidationError("must specify id for stored search template", validationException); + } else if (id.contains("#")) { + validationException = addValidationError("id cannot contain '#' for stored search template", validationException); + } + + return validationException; + } + + public String id() { + return id; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + id = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(id); + } + + @Override + public String toString() { + return "get stored search template [" + id + "]"; + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequestBuilder.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequestBuilder.java new file mode 100644 index 0000000000000..038c9b4b78c29 --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequestBuilder.java @@ -0,0 +1,33 @@ +/* + * 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.mustache.stored; + +import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; +import org.elasticsearch.client.ElasticsearchClient; + +public class GetStoredSearchTemplateRequestBuilder extends MasterNodeReadOperationRequestBuilder< + GetStoredSearchTemplateRequest, + GetStoredSearchTemplateResponse, + GetStoredSearchTemplateRequestBuilder> { + + public GetStoredSearchTemplateRequestBuilder(ElasticsearchClient client, GetStoredSearchTemplateAction action) { + super(client, action, new GetStoredSearchTemplateRequest()); + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateResponse.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateResponse.java new file mode 100644 index 0000000000000..284935faa1a95 --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateResponse.java @@ -0,0 +1,78 @@ +/* + * 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.mustache.stored; + +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.script.StoredScriptSource; + +import java.io.IOException; + +public class GetStoredSearchTemplateResponse extends ActionResponse implements ToXContent { + + private StoredScriptSource source; + + GetStoredSearchTemplateResponse() { + } + + GetStoredSearchTemplateResponse(StoredScriptSource source) { + this.source = source; + } + + /** + * @return if a stored script and if not found null + */ + public StoredScriptSource getSource() { + return source; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + source.toXContent(builder, params); + + return builder; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + + if (in.readBoolean()) { + source = new StoredScriptSource(in); + } else { + source = null; + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + + if (source == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + source.writeTo(out); + } + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateAction.java new file mode 100644 index 0000000000000..e67716f8e4480 --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateAction.java @@ -0,0 +1,43 @@ +/* + * 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.mustache.stored; + +import org.elasticsearch.action.Action; +import org.elasticsearch.client.ElasticsearchClient; + +public class PutStoredSearchTemplateAction + extends Action { + public static final PutStoredSearchTemplateAction INSTANCE = new PutStoredSearchTemplateAction(); + public static final String NAME = "cluster:admin/search/template/put"; + + private PutStoredSearchTemplateAction() { + super(NAME); + } + + @Override + public PutStoredSearchTemplateResponse newResponse() { + return new PutStoredSearchTemplateResponse(); + } + + @Override + public PutStoredSearchTemplateRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new PutStoredSearchTemplateRequestBuilder(client, this); + } +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequest.java similarity index 88% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequest.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequest.java index 9bd74346aca49..46d91f7d27360 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.script.mustache; +package org.elasticsearch.script.mustache.stored; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.support.master.AcknowledgedRequest; @@ -32,16 +32,16 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public class PutSearchTemplateRequest extends AcknowledgedRequest { +public class PutStoredSearchTemplateRequest extends AcknowledgedRequest { private String id; private BytesReference content; private XContentType xContentType; - public PutSearchTemplateRequest() { + public PutStoredSearchTemplateRequest() { super(); } - public PutSearchTemplateRequest(String id, BytesReference content, XContentType xContentType) { + public PutStoredSearchTemplateRequest(String id, BytesReference content, XContentType xContentType) { super(); this.id = id; this.content = content; @@ -55,11 +55,11 @@ public ActionRequestValidationException validate() { if (id == null || id.isEmpty()) { validationException = addValidationError("must specify id for stored search template", validationException); } else if (id.contains("#")) { - validationException = addValidationError("id cannot contain '#' for search template", validationException); + validationException = addValidationError("id cannot contain '#' for stored search template", validationException); } if (content == null) { - validationException = addValidationError("must specify code for search template", validationException); + validationException = addValidationError("must specify code for stored search template", validationException); } return validationException; diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequestBuilder.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequestBuilder.java similarity index 66% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequestBuilder.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequestBuilder.java index e61264469ed15..3637f85873eaa 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateRequestBuilder.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequestBuilder.java @@ -17,15 +17,17 @@ * under the License. */ -package org.elasticsearch.script.mustache; +package org.elasticsearch.script.mustache.stored; import org.elasticsearch.action.support.master.AcknowledgedRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; -public class PutSearchTemplateRequestBuilder - extends AcknowledgedRequestBuilder { +public class PutStoredSearchTemplateRequestBuilder extends AcknowledgedRequestBuilder< + PutStoredSearchTemplateRequest, + PutStoredSearchTemplateResponse, + PutStoredSearchTemplateRequestBuilder> { - public PutSearchTemplateRequestBuilder(ElasticsearchClient client, PutSearchTemplateAction action) { - super(client, action, new PutSearchTemplateRequest()); + public PutStoredSearchTemplateRequestBuilder(ElasticsearchClient client, PutStoredSearchTemplateAction action) { + super(client, action, new PutStoredSearchTemplateRequest()); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateResponse.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateResponse.java similarity index 85% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateResponse.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateResponse.java index d7665ede89d2a..a28f2b60f5e22 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/PutSearchTemplateResponse.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateResponse.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.script.mustache; +package org.elasticsearch.script.mustache.stored; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.common.io.stream.StreamInput; @@ -25,11 +25,11 @@ import java.io.IOException; -public class PutSearchTemplateResponse extends AcknowledgedResponse { - PutSearchTemplateResponse() { +public class PutStoredSearchTemplateResponse extends AcknowledgedResponse { + PutStoredSearchTemplateResponse() { } - public PutSearchTemplateResponse(boolean acknowledged) { + public PutStoredSearchTemplateResponse(boolean acknowledged) { super(acknowledged); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestDeleteSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredSearchTemplateAction.java similarity index 71% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestDeleteSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredSearchTemplateAction.java index 61a394462f9c8..e897b221035b0 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestDeleteSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredSearchTemplateAction.java @@ -16,24 +16,22 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.script.mustache; +package org.elasticsearch.script.mustache.stored; -import org.elasticsearch.action.admin.cluster.storedscripts.DeleteStoredScriptRequest; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.AcknowledgedRestListener; -import org.elasticsearch.script.Script; import java.io.IOException; import static org.elasticsearch.rest.RestRequest.Method.DELETE; -public class RestDeleteSearchTemplateAction extends BaseRestHandler { +public class RestDeleteStoredSearchTemplateAction extends BaseRestHandler { - public RestDeleteSearchTemplateAction(Settings settings, RestController controller) { + public RestDeleteStoredSearchTemplateAction(Settings settings, RestController controller) { super(settings); controller.registerHandler(DELETE, "/_search/template/{id}", this); @@ -43,7 +41,7 @@ public RestDeleteSearchTemplateAction(Settings settings, RestController controll public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { String id = request.param("id"); - DeleteStoredScriptRequest deleteStoredScriptRequest = new DeleteStoredScriptRequest(id, Script.DEFAULT_TEMPLATE_LANG); - return channel -> client.admin().cluster().deleteStoredScript(deleteStoredScriptRequest, new AcknowledgedRestListener<>(channel)); + DeleteStoredSearchTemplateRequest deleteRequest = new DeleteStoredSearchTemplateRequest(id); + return channel -> client.execute(DeleteStoredSearchTemplateAction.INSTANCE, deleteRequest, new AcknowledgedRestListener<>(channel)); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestGetSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredSearchTemplateAction.java similarity index 56% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestGetSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredSearchTemplateAction.java index 7d1ed4b57a421..76c71b6ec36ea 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestGetSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredSearchTemplateAction.java @@ -16,10 +16,8 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.script.mustache; +package org.elasticsearch.script.mustache.stored; -import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptRequest; -import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptResponse; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.settings.Settings; @@ -38,13 +36,13 @@ import static org.elasticsearch.rest.RestRequest.Method.GET; -public class RestGetSearchTemplateAction extends BaseRestHandler { +public class RestGetStoredSearchTemplateAction extends BaseRestHandler { public static final ParseField _ID_PARSE_FIELD = new ParseField("_id"); public static final ParseField FOUND_PARSE_FIELD = new ParseField("found"); - public RestGetSearchTemplateAction(Settings settings, RestController controller) { + public RestGetStoredSearchTemplateAction(Settings settings, RestController controller) { super(settings); controller.registerHandler(GET, "/_search/template/{id}", this); @@ -54,28 +52,29 @@ public RestGetSearchTemplateAction(Settings settings, RestController controller) public RestChannelConsumer prepareRequest(final RestRequest request, NodeClient client) throws IOException { String id = request.param("id"); - GetStoredScriptRequest getRequest = new GetStoredScriptRequest(id, Script.DEFAULT_TEMPLATE_LANG); + GetStoredSearchTemplateRequest getRequest = new GetStoredSearchTemplateRequest(id); - return channel -> client.admin().cluster().getStoredScript(getRequest, new RestBuilderListener(channel) { - @Override - public RestResponse buildResponse(GetStoredScriptResponse response, XContentBuilder builder) throws Exception { - builder.startObject(); - builder.field(_ID_PARSE_FIELD.getPreferredName(), id); + return channel -> client.execute(GetStoredSearchTemplateAction.INSTANCE, getRequest, + new RestBuilderListener(channel) { + @Override + public RestResponse buildResponse(GetStoredSearchTemplateResponse response, XContentBuilder builder) throws Exception { + builder.startObject(); + builder.field(_ID_PARSE_FIELD.getPreferredName(), id); - builder.field(StoredScriptSource.LANG_PARSE_FIELD.getPreferredName(), Script.DEFAULT_TEMPLATE_LANG); + builder.field(StoredScriptSource.LANG_PARSE_FIELD.getPreferredName(), Script.DEFAULT_TEMPLATE_LANG); - StoredScriptSource source = response.getSource(); - boolean found = source != null; - builder.field(FOUND_PARSE_FIELD.getPreferredName(), found); + StoredScriptSource source = response.getSource(); + boolean found = source != null; + builder.field(FOUND_PARSE_FIELD.getPreferredName(), found); - if (found) { - builder.field(StoredScriptSource.TEMPLATE_PARSE_FIELD.getPreferredName(), source.getCode()); - } + if (found) { + builder.field(StoredScriptSource.TEMPLATE_PARSE_FIELD.getPreferredName(), source.getCode()); + } - builder.endObject(); + builder.endObject(); - return new BytesRestResponse(found ? RestStatus.OK : RestStatus.NOT_FOUND, builder); - } - }); + return new BytesRestResponse(found ? RestStatus.OK : RestStatus.NOT_FOUND, builder); + } + }); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestPutSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredSearchTemplateAction.java similarity index 74% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestPutSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredSearchTemplateAction.java index 83925f0ec03b6..84950d9cebc26 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestPutSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredSearchTemplateAction.java @@ -16,9 +16,8 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.script.mustache; +package org.elasticsearch.script.mustache.stored; -import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptRequest; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; @@ -26,16 +25,15 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.AcknowledgedRestListener; -import org.elasticsearch.script.Script; import java.io.IOException; import static org.elasticsearch.rest.RestRequest.Method.POST; import static org.elasticsearch.rest.RestRequest.Method.PUT; -public class RestPutSearchTemplateAction extends BaseRestHandler { +public class RestPutStoredSearchTemplateAction extends BaseRestHandler { - public RestPutSearchTemplateAction(Settings settings, RestController controller) { + public RestPutStoredSearchTemplateAction(Settings settings, RestController controller) { super(settings); controller.registerHandler(POST, "/_search/template/{id}", this); @@ -47,7 +45,7 @@ public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client String id = request.param("id"); BytesReference content = request.content(); - PutStoredScriptRequest put = new PutStoredScriptRequest(id, Script.DEFAULT_TEMPLATE_LANG, content, request.getXContentType()); - return channel -> client.admin().cluster().putStoredScript(put, new AcknowledgedRestListener<>(channel)); + PutStoredSearchTemplateRequest put = new PutStoredSearchTemplateRequest(id, content, request.getXContentType()); + return channel -> client.execute(PutStoredSearchTemplateAction.INSTANCE, put, new AcknowledgedRestListener<>(channel)); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredSearchTemplateAction.java new file mode 100644 index 0000000000000..cd931b4cf2a9e --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredSearchTemplateAction.java @@ -0,0 +1,88 @@ +/* + * 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.mustache.stored; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.TransportMasterNodeAction; +import org.elasticsearch.cluster.AckedClusterStateUpdateTask; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.script.ScriptMetaData; +import org.elasticsearch.script.TemplateService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +public class TransportDeleteStoredSearchTemplateAction + extends TransportMasterNodeAction { + + private final TemplateService templateService; + + @Inject + public TransportDeleteStoredSearchTemplateAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, TemplateService templateService) { + super(settings, DeleteStoredSearchTemplateAction.NAME, transportService, clusterService, threadPool, actionFilters, + indexNameExpressionResolver, DeleteStoredSearchTemplateRequest::new); + this.templateService = templateService; + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected DeleteStoredSearchTemplateResponse newResponse() { + return new DeleteStoredSearchTemplateResponse(); + } + + @Override + protected void masterOperation(DeleteStoredSearchTemplateRequest request, ClusterState state, + ActionListener listener) throws Exception { + clusterService.submitStateUpdateTask("delete-search-template-" + request.id(), + new AckedClusterStateUpdateTask(request, listener) { + @Override + protected DeleteStoredSearchTemplateResponse newResponse(boolean acknowledged) { + return new DeleteStoredSearchTemplateResponse(acknowledged); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); + smd = ScriptMetaData.deleteStoredScript(smd, request.id(), templateService.getTemplateLanguage()); + MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); + return ClusterState.builder(currentState).metaData(mdb).build(); + } + }); + } + + @Override + protected ClusterBlockException checkBlock(DeleteStoredSearchTemplateRequest request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); + } + +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredSearchTemplateAction.java new file mode 100644 index 0000000000000..cc02a0d1c0300 --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredSearchTemplateAction.java @@ -0,0 +1,79 @@ +/* + * 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.mustache.stored; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.script.ScriptMetaData; +import org.elasticsearch.script.TemplateService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +public class TransportGetStoredSearchTemplateAction + extends TransportMasterNodeReadAction { + + private final TemplateService templateService; + + @Inject + public TransportGetStoredSearchTemplateAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, TemplateService templateService) { + super(settings, GetStoredSearchTemplateAction.NAME, transportService, clusterService, threadPool, actionFilters, + indexNameExpressionResolver, GetStoredSearchTemplateRequest::new); + this.templateService = templateService; + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected GetStoredSearchTemplateResponse newResponse() { + return new GetStoredSearchTemplateResponse(); + } + + @Override + protected void masterOperation(GetStoredSearchTemplateRequest request, ClusterState state, + ActionListener listener) throws Exception { + ScriptMetaData scriptMetadata = state.metaData().custom(ScriptMetaData.TYPE); + + if (scriptMetadata != null) { + listener.onResponse(new GetStoredSearchTemplateResponse( + scriptMetadata.getStoredScript(request.id(), templateService.getTemplateLanguage()))); + } else { + listener.onResponse(null); + } + } + + @Override + protected ClusterBlockException checkBlock(GetStoredSearchTemplateRequest request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ); + } + +} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java new file mode 100644 index 0000000000000..5cee30deb611b --- /dev/null +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java @@ -0,0 +1,109 @@ +/* + * 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.mustache.stored; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.TransportMasterNodeAction; +import org.elasticsearch.cluster.AckedClusterStateUpdateTask; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.script.ScriptException; +import org.elasticsearch.script.ScriptMetaData; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.StoredScriptSource; +import org.elasticsearch.script.TemplateService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +public class TransportPutStoredSearchTemplateAction + extends TransportMasterNodeAction { + + private final TemplateService templateService; + private final int maxScriptSizeInBytes; + + @Inject + public TransportPutStoredSearchTemplateAction(Settings settings, TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, TemplateService templateService) { + super(settings, PutStoredSearchTemplateAction.NAME, transportService, clusterService, threadPool, actionFilters, + indexNameExpressionResolver, PutStoredSearchTemplateRequest::new); + this.templateService = templateService; + maxScriptSizeInBytes = ScriptService.SCRIPT_MAX_SIZE_IN_BYTES.get(settings); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected PutStoredSearchTemplateResponse newResponse() { + return new PutStoredSearchTemplateResponse(); + } + + @Override + protected void masterOperation(PutStoredSearchTemplateRequest request, ClusterState state, + ActionListener listener) throws Exception { + if (request.content().length() > maxScriptSizeInBytes) { + throw new IllegalArgumentException("exceeded max allowed stored script size in bytes [" + maxScriptSizeInBytes + + "] with size [" + request.content().length() + "] for script [" + request.id() + "]"); + } + + StoredScriptSource source = StoredScriptSource.parse(templateService.getTemplateLanguage(), request.content(), + request.xContentType()); + try { + templateService.checkCompileBeforeStore(source); + } catch (IllegalArgumentException | ScriptException e) { + throw new IllegalArgumentException("failed to parse/compile stored search template [" + request.id() + "]" + + (source.getCode() == null ? "" : " using code [" + source.getCode() + "]"), e); + } + + clusterService.submitStateUpdateTask("put-search-template-" + request.id(), + new AckedClusterStateUpdateTask(request, listener) { + + @Override + protected PutStoredSearchTemplateResponse newResponse(boolean acknowledged) { + return new PutStoredSearchTemplateResponse(acknowledged); + } + + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); + smd = ScriptMetaData.putStoredScript(smd, request.id(), source); + MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); + + return ClusterState.builder(currentState).metaData(mdb).build(); + } + }); + } + + @Override + protected ClusterBlockException checkBlock(PutStoredSearchTemplateRequest request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); + } + +} diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/TemplateQueryBuilderTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/TemplateQueryBuilderTests.java index f3ed3ed68f206..04f3a341e27f0 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/TemplateQueryBuilderTests.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/TemplateQueryBuilderTests.java @@ -22,7 +22,6 @@ import org.apache.lucene.index.memory.MemoryIndex; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; @@ -33,7 +32,6 @@ import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.script.MockScriptPlugin; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptType; import org.elasticsearch.search.internal.SearchContext; @@ -42,13 +40,12 @@ import org.junit.Before; import java.io.IOException; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; -import java.util.function.Function; +import static java.util.Collections.singleton; import static org.hamcrest.Matchers.containsString; public class TemplateQueryBuilderTests extends AbstractQueryTestCase { @@ -69,38 +66,7 @@ public void checkWarning() { @Override protected Collection> getPlugins() { - return Arrays.asList(MustachePlugin.class, CustomScriptPlugin.class); - } - - public static class CustomScriptPlugin extends MockScriptPlugin { - - @Override - protected Map, Object>> pluginScripts() { - Map, Object>> scripts = new HashMap<>(); - - scripts.put("{ \"match_all\" : {}}", - s -> new BytesArray("{ \"match_all\" : {}}")); - - scripts.put("{ \"match_all\" : {\"_name\" : \"foobar\"}}", - s -> new BytesArray("{ \"match_all\" : {\"_name\" : \"foobar\"}}")); - - scripts.put("{\n" + - " \"term\" : {\n" + - " \"foo\" : {\n" + - " \"value\" : \"bar\",\n" + - " \"boost\" : 2.0\n" + - " }\n" + - " }\n" + - "}", s -> new BytesArray("{\n" + - " \"term\" : {\n" + - " \"foo\" : {\n" + - " \"value\" : \"bar\",\n" + - " \"boost\" : 2.0\n" + - " }\n" + - " }\n" + - "}")); - return scripts; - } + return singleton(MustachePlugin.class); } @Before diff --git a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java b/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java index 2c6176b8f6fa5..654b27c4dae82 100644 --- a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java +++ b/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java @@ -45,7 +45,7 @@ public void init() throws Exception { .build(); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Arrays.asList(new MustacheScriptEngineService())); ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); ScriptService scriptService = new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, scriptContextRegistry, scriptSettings, new ScriptMetrics()); From 085749be7a3079ed3f0924cd6ef212c03fa1b955 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sun, 5 Mar 2017 11:07:18 -0500 Subject: [PATCH 03/37] Closer --- .../java/org/elasticsearch/node/Node.java | 1 + .../elasticsearch/script/CachingCompiler.java | 11 +++-- .../elasticsearch/script/ScriptModule.java | 4 +- .../elasticsearch/script/TemplateService.java | 9 +++- .../script/mustache/MustachePlugin.java | 5 ++- .../DeleteStoredSearchTemplateAction.java | 2 +- .../DeleteStoredSearchTemplateRequest.java | 6 ++- ...eteStoredSearchTemplateRequestBuilder.java | 5 +++ .../stored/GetStoredSearchTemplateAction.java | 2 +- .../GetStoredSearchTemplateRequest.java | 6 ++- ...GetStoredSearchTemplateRequestBuilder.java | 5 +++ .../PutStoredSearchTemplateRequest.java | 13 +++++- ...PutStoredSearchTemplateRequestBuilder.java | 13 ++++++ .../script/mustache/SearchTemplateIT.java | 44 ++++++++----------- 14 files changed, 83 insertions(+), 43 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 3cfe4157c2395..8a8f8c9698326 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -333,6 +333,7 @@ protected Node(final Environment environment, Collection final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool, localNodeFactory::getNode); clusterService.addListener(scriptModule.getScriptService()); + clusterService.addListener(scriptModule.getTemplateService()); resourcesToClose.add(clusterService); final IngestService ingestService = new IngestService(settings, threadPool, this.environment, scriptModule.getScriptService(), analysisModule.getAnalysisRegistry(), pluginsService.filterPlugins(IngestPlugin.class)); diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 35c85b9b4329e..4a644f6ac18b1 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -90,7 +90,7 @@ public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environ } logger.debug("using script cache with max_size [{}], expire [{}]", cacheMaxSize, cacheExpire); - this.cache = cacheBuilder.removalListener(new ScriptCacheRemovalListener()).build(); + this.cache = cacheBuilder.removalListener(new CacheRemovalListener()).build(); // add file watcher for file scripts scriptsDirectory = env.scriptsFile(); @@ -98,7 +98,7 @@ public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environ logger.trace("Using scripts directory [{}] ", scriptsDirectory); } FileWatcher fileWatcher = new FileWatcher(scriptsDirectory); - fileWatcher.addListener(new ScriptsDirectoryChangesListener()); + fileWatcher.addListener(new DirectoryChangesListener()); if (ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.get(settings)) { // automatic reload is enabled - register scripts resourceWatcherService.add(fileWatcher); @@ -112,7 +112,7 @@ public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environ /** * Build the cache key for a file name and its extension. Return null to indicate that the file type is not supported. - */ // NOCOMMIT rename to cacheKeyForFile + */ protected abstract CacheKeyT cacheKeyForFile(String baseName, String extension); protected abstract CacheKeyT cacheKeyFromClusterState(StoredScriptSource scriptMetadata); protected abstract StoredScriptSource lookupStoredScript(ClusterState clusterState, CacheKeyT cacheKey); @@ -192,7 +192,6 @@ private CacheKeyT getScriptFromClusterState(CacheKeyT cacheKey) { if (source == null) { throw new ResourceNotFoundException("unable to find script [" + cacheKey + "] in cluster state"); } - return cacheKeyFromClusterState(source); } @@ -226,7 +225,7 @@ public final void clusterChanged(ClusterChangedEvent event) { /** * Listener to manage metrics for the script cache. */ - private class ScriptCacheRemovalListener implements RemovalListener { + private class CacheRemovalListener implements RemovalListener { @Override public void onRemoval(RemovalNotification notification) { if (logger.isDebugEnabled()) { @@ -236,7 +235,7 @@ public void onRemoval(RemovalNotification notificatio } } - private class ScriptsDirectoryChangesListener implements FileChangesListener { + private class DirectoryChangesListener implements FileChangesListener { private Tuple getScriptNameExt(Path file) { Path scriptPath = scriptsDirectory.relativize(file); int extIndex = scriptPath.toString().lastIndexOf('.'); diff --git a/core/src/main/java/org/elasticsearch/script/ScriptModule.java b/core/src/main/java/org/elasticsearch/script/ScriptModule.java index 4b07b80dd7368..54c1699fe3f39 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptModule.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptModule.java @@ -86,12 +86,14 @@ public ScriptModule(Settings settings, Environment environment, ScriptMetrics scriptMetrics = new ScriptMetrics(); // Note that if templateBackend is null this won't register any settings for it scriptSettings = new ScriptSettings(scriptEngineRegistry, templateBackend, scriptContextRegistry); + try { scriptService = new ScriptService(settings, environment, resourceWatcherService, scriptEngineRegistry, scriptContextRegistry, scriptSettings, scriptMetrics); } catch (IOException e) { throw new RuntimeException("Couldn't setup ScriptService", e); } + if (templateBackend == null) { templateBackend = new TemplatesUnsupportedBackend(); } @@ -99,7 +101,7 @@ public ScriptModule(Settings settings, Environment environment, templateService = new TemplateService(settings, environment, resourceWatcherService, templateBackend, scriptContextRegistry, scriptSettings, scriptMetrics); } catch (IOException e) { - throw new RuntimeException("Couldn't setup ScriptService", e); + throw new RuntimeException("Couldn't setup TemplateService", e); } } diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index 6e0dc44d2dfd9..d49137fd3bbd4 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -20,7 +20,9 @@ package org.elasticsearch.script; import org.apache.logging.log4j.Logger; +import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.settings.Settings; @@ -34,7 +36,7 @@ import static java.util.Collections.emptyMap; -public class TemplateService { +public class TemplateService implements ClusterStateListener { public interface Backend extends ScriptEngineService {} // NOCOMMIT this should diverge.... private static final Logger logger = ESLoggerFactory.getLogger(TemplateService.class); @@ -142,4 +144,9 @@ public String getTemplateLanguage() { public void checkCompileBeforeStore(StoredScriptSource source) { compiler.checkCompileBeforeStore(source); } + + @Override + public void clusterChanged(ClusterChangedEvent event) { + compiler.clusterChanged(event); + } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java index 0ee2943a83cfb..95f77de8635a9 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java @@ -36,11 +36,13 @@ import org.elasticsearch.script.TemplateService.Backend; import org.elasticsearch.script.mustache.stored.DeleteStoredSearchTemplateAction; import org.elasticsearch.script.mustache.stored.GetStoredSearchTemplateAction; +import org.elasticsearch.script.mustache.stored.PutStoredSearchTemplateAction; import org.elasticsearch.script.mustache.stored.RestDeleteStoredSearchTemplateAction; import org.elasticsearch.script.mustache.stored.RestGetStoredSearchTemplateAction; import org.elasticsearch.script.mustache.stored.RestPutStoredSearchTemplateAction; import org.elasticsearch.script.mustache.stored.TransportDeleteStoredSearchTemplateAction; import org.elasticsearch.script.mustache.stored.TransportGetStoredSearchTemplateAction; +import org.elasticsearch.script.mustache.stored.TransportPutStoredSearchTemplateAction; import java.util.Arrays; import java.util.List; @@ -60,7 +62,8 @@ public Backend getTemplateBackend() { new ActionHandler<>(SearchTemplateAction.INSTANCE, TransportSearchTemplateAction.class), new ActionHandler<>(MultiSearchTemplateAction.INSTANCE, TransportMultiSearchTemplateAction.class), new ActionHandler<>(DeleteStoredSearchTemplateAction.INSTANCE, TransportDeleteStoredSearchTemplateAction.class), - new ActionHandler<>(GetStoredSearchTemplateAction.INSTANCE, TransportGetStoredSearchTemplateAction.class)); + new ActionHandler<>(GetStoredSearchTemplateAction.INSTANCE, TransportGetStoredSearchTemplateAction.class), + new ActionHandler<>(PutStoredSearchTemplateAction.INSTANCE, TransportPutStoredSearchTemplateAction.class)); } @Override diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java index 1e511d2ed0009..770007d6c77b4 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java @@ -26,7 +26,7 @@ public class DeleteStoredSearchTemplateAction extends Action { public static final DeleteStoredSearchTemplateAction INSTANCE = new DeleteStoredSearchTemplateAction(); - public static final String NAME = "cluster:admin/script/delete"; + public static final String NAME = "cluster:admin/search/template/delete"; private DeleteStoredSearchTemplateAction() { super(NAME); diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java index ed3958f15291d..d121f07b66e5c 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java @@ -32,13 +32,15 @@ public class DeleteStoredSearchTemplateRequest extends AcknowledgedRequest { public static final GetStoredSearchTemplateAction INSTANCE = new GetStoredSearchTemplateAction(); - public static final String NAME = "cluster:admin/script/get"; + public static final String NAME = "cluster:admin/search/template/get"; private GetStoredSearchTemplateAction() { super(NAME); diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java index 3d2ad61fba5bf..1b36cce049d12 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java @@ -32,13 +32,15 @@ public class GetStoredSearchTemplateRequest extends MasterNodeReadRequest new SearchTemplateRequestBuilder(client()) @@ -211,8 +211,7 @@ public void testIndexedTemplateClient() throws Exception { } public void testIndexedTemplate() throws Exception { - assertAcked(client().admin().cluster().preparePutStoredScript() - .setLang(MustacheScriptEngineService.NAME) + assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("1a") .setContent(new BytesArray("{" + "\"template\":{" + @@ -224,8 +223,7 @@ public void testIndexedTemplate() throws Exception { "}" ), XContentType.JSON) ); - assertAcked(client().admin().cluster().preparePutStoredScript() - .setLang(MustacheScriptEngineService.NAME) + assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("2") .setContent(new BytesArray("{" + "\"template\":{" + @@ -236,8 +234,7 @@ public void testIndexedTemplate() throws Exception { "}" + "}"), XContentType.JSON) ); - assertAcked(client().admin().cluster().preparePutStoredScript() - .setLang(MustacheScriptEngineService.NAME) + assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("3") .setContent(new BytesArray("{" + "\"template\":{" + @@ -312,8 +309,7 @@ public void testIndexedTemplateOverwrite() throws Exception { int iterations = randomIntBetween(2, 11); for (int i = 1; i < iterations; i++) { - assertAcked(client().admin().cluster().preparePutStoredScript() - .setLang(MustacheScriptEngineService.NAME) + assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("git01") .setContent(new BytesArray("{\"template\":{\"query\": {\"match\": {\"searchtext\": {\"query\": \"{{P_Keyword1}}\"," + "\"type\": \"ooophrase_prefix\"}}}}}"), XContentType.JSON)); @@ -332,8 +328,7 @@ public void testIndexedTemplateOverwrite() throws Exception { assertThat(e.getMessage(), containsString("[match] query does not support type ooophrase_prefix")); assertWarnings("Deprecated field [type] used, replaced by [match_phrase and match_phrase_prefix query]"); - assertAcked(client().admin().cluster().preparePutStoredScript() - .setLang(MustacheScriptEngineService.NAME) + assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("git01") .setContent(new BytesArray("{\"query\": {\"match\": {\"searchtext\": {\"query\": \"{{P_Keyword1}}\"," + "\"type\": \"phrase_prefix\"}}}}"), XContentType.JSON)); @@ -349,12 +344,9 @@ public void testIndexedTemplateOverwrite() throws Exception { public void testIndexedTemplateWithArray() throws Exception { String multiQuery = "{\"query\":{\"terms\":{\"theField\":[\"{{#fieldParam}}\",\"{{.}}\",\"{{/fieldParam}}\"]}}}"; - assertAcked( - client().admin().cluster().preparePutStoredScript() - .setLang(MustacheScriptEngineService.NAME) - .setId("4") - .setContent(jsonBuilder().startObject().field("template", multiQuery).endObject().bytes(), XContentType.JSON) - ); + assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) + .setId("4") + .setContent(jsonBuilder().startObject().field("template", multiQuery).endObject().bytes(), XContentType.JSON)); BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); bulkRequestBuilder.add(client().prepareIndex("test", "type", "1").setSource("{\"theField\":\"foo\"}", XContentType.JSON)); bulkRequestBuilder.add(client().prepareIndex("test", "type", "2").setSource("{\"theField\":\"foo 2\"}", XContentType.JSON)); From b5f16fdfe0d6f8adc42f774cb4b04efde7de9dcd Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sun, 12 Mar 2017 16:09:47 -0700 Subject: [PATCH 04/37] Something I forgot:wq --- .../org/elasticsearch/common/xcontent/XContentFactory.java | 2 +- .../org/elasticsearch/index/query/QueryRewriteContext.java | 4 ++-- core/src/main/java/org/elasticsearch/node/Node.java | 2 +- .../java/org/elasticsearch/script/CachingCompiler.java | 6 ++++-- .../java/org/elasticsearch/script/TemplateService.java | 7 +++++-- .../script/mustache/MustacheScriptEngineService.java | 1 + .../script/mustache/TransportSearchTemplateAction.java | 5 +---- 7 files changed, 15 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java b/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java index dc9d1c493a3ed..4bb45055f5b85 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java @@ -278,7 +278,7 @@ public static XContentType xContentType(byte[] data, int offset, int length) { public static XContent xContent(BytesReference bytes) { XContentType type = xContentType(bytes); if (type == null) { - throw new ElasticsearchParseException("Failed to derive xcontent"); + throw new ElasticsearchParseException("Failed to derive xcontent for [" + bytes.utf8ToString() + "]"); // NOCOMMIT revert me. } return xContent(type); } diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java index 4463cae006700..27d5c294fe5be 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java @@ -21,6 +21,7 @@ import org.apache.lucene.index.IndexReader; import org.elasticsearch.client.Client; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.IndexSettings; @@ -107,9 +108,8 @@ public long nowInMillis() { } public BytesReference getTemplateBytes(Script template) { - ExecutableScript executable = templateService.executable(template.getIdOrCode(), template.getType(), ScriptContext.Standard.SEARCH, + return templateService.render(template.getIdOrCode(), template.getType(), ScriptContext.Standard.SEARCH, template.getParams()); - return (BytesReference) executable.run(); } public TemplateService getTemplateService() { diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 8a8f8c9698326..bca6ef9685c0a 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -332,7 +332,7 @@ protected Node(final Environment environment, Collection getCustomNameResolvers(pluginsService.filterPlugins(DiscoveryPlugin.class))); final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool, localNodeFactory::getNode); - clusterService.addListener(scriptModule.getScriptService()); + clusterService.addListener(scriptModule.getScriptService()); // NOCOMMIT why not do this in the service's ctor? clusterService.addListener(scriptModule.getTemplateService()); resourcesToClose.add(clusterService); final IngestService ingestService = new IngestService(settings, threadPool, this.environment, diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 4a644f6ac18b1..76f6c6e09ed9a 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -41,6 +41,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.env.Environment; +import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.watcher.FileChangesListener; import org.elasticsearch.watcher.FileWatcher; import org.elasticsearch.watcher.ResourceWatcherService; @@ -78,6 +79,7 @@ public abstract class CachingCompiler implements ClusterStateListener public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environment env, ResourceWatcherService resourceWatcherService, ScriptMetrics scriptMetrics) throws IOException { int cacheMaxSize = ScriptService.SCRIPT_CACHE_SIZE_SETTING.get(settings); + this.scriptMetrics = scriptMetrics; CacheBuilder cacheBuilder = CacheBuilder.builder(); if (cacheMaxSize >= 0) { @@ -106,8 +108,6 @@ public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environ // automatic reload is disabled just load scripts once fileWatcher.init(); } - - this.scriptMetrics = scriptMetrics; } /** @@ -134,6 +134,7 @@ public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environ protected abstract CompiledScript compileFileScript(CacheKeyT cacheKey, String body, Path file); public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, ScriptContext scriptContext) { + ESLoggerFactory.getLogger(QueryRewriteContext.class).warn("ASDFASDF get {} {}", cacheKey, scriptType); Objects.requireNonNull(cacheKey); // First resolve stored scripts so so we have accurate parameters for checkCanExecuteScript @@ -150,6 +151,7 @@ public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, if (compiled == null) { throw new IllegalArgumentException("unable to find file script " + cacheKey); } + ESLoggerFactory.getLogger(QueryRewriteContext.class).warn("ASDFASDF got file {}", compiled); return compiled; } diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index d49137fd3bbd4..25211911cfc7a 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -23,6 +23,7 @@ import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.settings.Settings; @@ -129,9 +130,11 @@ protected CompiledScript compileFileScript(String cacheKey, String body, Path fi }; } - public ExecutableScript executable(String idOrCode, ScriptType scriptType, ScriptContext scriptContext, + public BytesReference render(String idOrCode, ScriptType scriptType, ScriptContext scriptContext, Map scriptParams) { - return backend.executable(compiler.getScript(idOrCode, scriptType, scriptContext), scriptParams); + BytesReference b = (BytesReference) backend.executable(compiler.getScript(idOrCode, scriptType, scriptContext), scriptParams).run(); + ESLoggerFactory.getLogger(TemplateService.class).warn("ASDFASDF rendered [{}]", b.utf8ToString()); + return b; } /** diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java index 48b652d0fefc4..7d7368b577867 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java @@ -107,6 +107,7 @@ public String getExtension() { @Override public ExecutableScript executable(CompiledScript compiledScript, @Nullable Map vars) { + ESLoggerFactory.getLogger(TemplateService.class).warn("ASDFASDF toexec [{}] [{}]", compiledScript, vars); return new MustacheExecutableScript(compiledScript, vars); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java index 7ae9ccab93ecd..1a58093dfa1c3 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java @@ -35,7 +35,6 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.script.ExecutableScript; -import org.elasticsearch.script.Script; import org.elasticsearch.script.TemplateService; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.threadpool.ThreadPool; @@ -66,10 +65,8 @@ public TransportSearchTemplateAction(Settings settings, ThreadPool threadPool, T protected void doExecute(SearchTemplateRequest request, ActionListener listener) { final SearchTemplateResponse response = new SearchTemplateResponse(); try { - ExecutableScript executable = templateService.executable(request.getScript(), request.getScriptType(), SEARCH, + BytesReference source = templateService.render(request.getScript(), request.getScriptType(), SEARCH, request.getScriptParams() == null ? Collections.emptyMap() : request.getScriptParams()); - - BytesReference source = (BytesReference) executable.run(); response.setSource(source); if (request.isSimulate()) { From 58a5ad991a8a8dc43b0dc049eff17f9f4b5b55a8 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sun, 19 Mar 2017 21:15:47 -0400 Subject: [PATCH 05/37] Line length --- .../elasticsearch/script/CachingCompiler.java | 99 ++++++++++++------- .../elasticsearch/script/ScriptPermits.java | 42 +++++--- .../elasticsearch/script/TemplateService.java | 58 +++++++---- .../script/ScriptPermitsTests.java | 3 +- .../script/TemplateServiceTests.java | 10 +- .../DeleteStoredSearchTemplateAction.java | 9 +- .../DeleteStoredSearchTemplateRequest.java | 4 +- ...eteStoredSearchTemplateRequestBuilder.java | 9 +- .../stored/GetStoredSearchTemplateAction.java | 7 +- .../GetStoredSearchTemplateRequest.java | 4 +- ...GetStoredSearchTemplateRequestBuilder.java | 9 +- .../stored/PutStoredSearchTemplateAction.java | 10 +- .../PutStoredSearchTemplateRequest.java | 23 +++-- ...PutStoredSearchTemplateRequestBuilder.java | 6 +- .../RestDeleteStoredSearchTemplateAction.java | 6 +- .../RestGetStoredSearchTemplateAction.java | 20 ++-- .../RestPutStoredSearchTemplateAction.java | 9 +- ...sportDeleteStoredSearchTemplateAction.java | 32 +++--- ...ransportGetStoredSearchTemplateAction.java | 25 ++--- ...ransportPutStoredSearchTemplateAction.java | 65 ++++++------ 20 files changed, 280 insertions(+), 170 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 76f6c6e09ed9a..a1dadd09cd614 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -55,7 +55,8 @@ import java.util.concurrent.ConcurrentMap; /** - * Manages caching, resource watching, permissions checking, and compilation of scripts (or templates). + * Manages caching, resource watching, permissions checking, and compilation of scripts (or + * templates). */ public abstract class CachingCompiler implements ClusterStateListener { private static final Logger logger = ESLoggerFactory.getLogger(CachingCompiler.class); @@ -63,7 +64,8 @@ public abstract class CachingCompiler implements ClusterStateListener /** * Compiled file scripts (or templates). Modified by the file watching process. */ - private final ConcurrentMap fileScripts = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap fileScripts = ConcurrentCollections + .newConcurrentMap(); /** * Cache of compiled dynamic scripts (or templates). @@ -77,7 +79,8 @@ public abstract class CachingCompiler implements ClusterStateListener private volatile ClusterState clusterState; public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environment env, - ResourceWatcherService resourceWatcherService, ScriptMetrics scriptMetrics) throws IOException { + ResourceWatcherService resourceWatcherService, ScriptMetrics scriptMetrics) + throws IOException { int cacheMaxSize = ScriptService.SCRIPT_CACHE_SIZE_SETTING.get(settings); this.scriptMetrics = scriptMetrics; @@ -91,9 +94,10 @@ public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environ cacheBuilder.setExpireAfterAccess(cacheExpire); } - logger.debug("using script cache with max_size [{}], expire [{}]", cacheMaxSize, cacheExpire); + logger.debug("using script cache with max_size [{}], expire [{}]", cacheMaxSize, + cacheExpire); this.cache = cacheBuilder.removalListener(new CacheRemovalListener()).build(); - + // add file watcher for file scripts scriptsDirectory = env.scriptsFile(); if (logger.isTraceEnabled()) { @@ -111,30 +115,42 @@ public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environ } /** - * Build the cache key for a file name and its extension. Return null to indicate that the file type is not supported. + * Build the cache key for a file name and its extension. Return null to indicate that the file + * type is not supported. */ protected abstract CacheKeyT cacheKeyForFile(String baseName, String extension); protected abstract CacheKeyT cacheKeyFromClusterState(StoredScriptSource scriptMetadata); - protected abstract StoredScriptSource lookupStoredScript(ClusterState clusterState, CacheKeyT cacheKey); + + protected abstract StoredScriptSource lookupStoredScript(ClusterState clusterState, + CacheKeyT cacheKey); + /** - * Are any script contexts enabled for the given {@code cacheKey} and {@code scriptType}? Used to reject compilation if all script - * contexts are disabled and produce a nice error message earlier rather than later. - */ // NOCOMMIT make sure we have tests for cases where we use this (known cases are files and cluster state) - protected abstract boolean areAnyScriptContextsEnabled(CacheKeyT cacheKey, ScriptType scriptType); + * Are any script contexts enabled for the given {@code cacheKey} and {@code scriptType}? Used + * to reject compilation if all script contexts are disabled and produce a nice error message + * earlier rather than later. + */ // NOCOMMIT make sure we have tests for cases where we use this (files, cluster state, ?) + protected abstract boolean areAnyScriptContextsEnabled(CacheKeyT cacheKey, + ScriptType scriptType); + /** * Check if a script can be executed. */ - protected abstract void checkCanExecuteScript(CacheKeyT cacheKey, ScriptType scriptType, ScriptContext scriptContext); + protected abstract void checkCanExecuteScript(CacheKeyT cacheKey, ScriptType scriptType, + ScriptContext scriptContext); + /** * Check if too many scripts (or templates) have been compiled recently. */ protected abstract void checkCompilationLimit(); + // NOCOMMIT document protected abstract CompiledScript compile(ScriptType scriptType, CacheKeyT cacheKey); protected abstract CompiledScript compileFileScript(CacheKeyT cacheKey, String body, Path file); - public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, ScriptContext scriptContext) { - ESLoggerFactory.getLogger(QueryRewriteContext.class).warn("ASDFASDF get {} {}", cacheKey, scriptType); + public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, + ScriptContext scriptContext) { + ESLoggerFactory.getLogger(QueryRewriteContext.class).warn("ASDFASDF get {} {}", cacheKey, + scriptType); Objects.requireNonNull(cacheKey); // First resolve stored scripts so so we have accurate parameters for checkCanExecuteScript @@ -151,7 +167,8 @@ public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, if (compiled == null) { throw new IllegalArgumentException("unable to find file script " + cacheKey); } - ESLoggerFactory.getLogger(QueryRewriteContext.class).warn("ASDFASDF got file {}", compiled); + ESLoggerFactory.getLogger(QueryRewriteContext.class).warn("ASDFASDF got file {}", + compiled); return compiled; } @@ -161,7 +178,8 @@ public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, return compiledScript; } - // Synchronize so we don't compile scripts many times during multiple shards all compiling a script + /* Synchronize so we don't compile scripts many times during multiple shards all compiling + * a script */ synchronized (this) { // Double check in case it was compiled while we were waiting for the monitor compiledScript = cache.get(cacheKey); @@ -177,7 +195,7 @@ public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, checkCompilationLimit(); compiledScript = compile(scriptType, cacheKey); } catch (ScriptException good) { - // TODO: remove this try-catch completely, when all script engines have good exceptions! + // TODO: remove this try-catch when all script engines have good exceptions! throw good; // its already good } catch (Exception exception) { throw new GeneralScriptException("Failed to compile " + cacheKey, exception); @@ -192,7 +210,8 @@ private CacheKeyT getScriptFromClusterState(CacheKeyT cacheKey) { StoredScriptSource source = lookupStoredScript(clusterState, cacheKey); if (source == null) { - throw new ResourceNotFoundException("unable to find script [" + cacheKey + "] in cluster state"); + throw new ResourceNotFoundException( + "unable to find script [" + cacheKey + "] in cluster state"); } return cacheKeyFromClusterState(source); } @@ -231,7 +250,8 @@ private class CacheRemovalListener implements RemovalListener notification) { if (logger.isDebugEnabled()) { - logger.debug("removed {} from cache, reason: {}", notification.getValue(), notification.getRemovalReason()); + logger.debug("removed {} from cache, reason: {}", notification.getValue(), + notification.getRemovalReason()); } scriptMetrics.onCacheEviction(); } @@ -250,7 +270,8 @@ private Tuple getScriptNameExt(Path file) { return null; } - String scriptName = scriptPath.toString().substring(0, extIndex).replace(scriptPath.getFileSystem().getSeparator(), "_"); + String scriptName = scriptPath.toString().substring(0, extIndex) + .replace(scriptPath.getFileSystem().getSeparator(), "_"); return new Tuple<>(scriptName, ext); } @@ -271,39 +292,45 @@ public void onFileInit(Path file) { return; } try { - /* we don't know yet what the script will be used for, but if all of the operations for this lang with file scripts are - * disabled, it makes no sense to even compile it and cache it. */ + /* we don't know yet what the script will be used for, but if all of the operations + * for this lang with file scripts are disabled, it makes no sense to even compile + * it and cache it. */ if (areAnyScriptContextsEnabled(cacheKey, ScriptType.FILE)) { logger.info("compiling script file [{}]", file.toAbsolutePath()); - try (InputStreamReader reader = new InputStreamReader(Files.newInputStream(file), StandardCharsets.UTF_8)) { + try (InputStreamReader reader = new InputStreamReader( + Files.newInputStream(file), StandardCharsets.UTF_8)) { String body = Streams.copyToString(reader); fileScripts.put(cacheKey, compileFileScript(cacheKey, body, file)); scriptMetrics.onCompilation(); } } else { - logger.warn("skipping compile of script file [{}] as all scripted operations are disabled for file scripts", - - file.toAbsolutePath()); + logger.warn("skipping compile of script file [{}] as all scripted operations " + + "are disabled for file scripts", file.toAbsolutePath()); } } catch (ScriptException e) { - /* Attempt to extract a concise error message using the xcontent generation mechanisms and log that. */ + /* Attempt to extract a concise error message using the xcontent generation + * mechanisms and log that. */ try (XContentBuilder builder = JsonXContent.contentBuilder()) { builder.prettyPrint(); builder.startObject(); - ElasticsearchException.generateThrowableXContent(builder, ToXContent.EMPTY_PARAMS, e); + ElasticsearchException.generateThrowableXContent(builder, + ToXContent.EMPTY_PARAMS, e); builder.endObject(); - logger.warn("failed to load/compile script [{}]: {}", scriptNameExt.v1(), builder.string()); + logger.warn("failed to load/compile script [{}]: {}", scriptNameExt.v1(), + builder.string()); } catch (IOException ioe) { ioe.addSuppressed(e); - logger.warn((Supplier) () -> new ParameterizedMessage( - "failed to log an appropriate warning after failing to load/compile script [{}]", scriptNameExt.v1()), ioe); + logger.warn((Supplier) () -> new ParameterizedMessage("failed to log an " + + "appropriate warning after failing to load/compile script [{}]", + scriptNameExt.v1()), ioe); } - /* Log at the whole exception at the debug level as well just in case the stack trace is important. That way you can - * turn on the stack trace if you need it. */ - logger.debug((Supplier) () -> new ParameterizedMessage("failed to load/compile script [{}]. full exception:", - scriptNameExt.v1()), e); + /* Log at the whole exception at the debug level as well just in case the stack + * trace is important. That way you can turn on the stack trace if you need it. */ + logger.debug((Supplier) () -> new ParameterizedMessage("failed to load/compile " + + "script [{}]. full exception:", scriptNameExt.v1()), e); } catch (Exception e) { - logger.warn((Supplier) () -> new ParameterizedMessage("failed to load/compile script [{}]", scriptNameExt.v1()), e); + logger.warn((Supplier) () -> new ParameterizedMessage("failed to load/compile " + + "script [{}]", scriptNameExt.v1()), e); } } diff --git a/core/src/main/java/org/elasticsearch/script/ScriptPermits.java b/core/src/main/java/org/elasticsearch/script/ScriptPermits.java index a6c1f90b5c5ec..bfe86cdf834b1 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptPermits.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptPermits.java @@ -40,32 +40,39 @@ public class ScriptPermits { private double scriptsPerMinCounter; private double compilesAllowedPerNano; - public ScriptPermits(Settings settings, ScriptSettings scriptSettings, ScriptContextRegistry scriptContextRegistry) { + public ScriptPermits(Settings settings, ScriptSettings scriptSettings, + ScriptContextRegistry scriptContextRegistry) { this.scriptModes = new ScriptModes(scriptSettings, settings); this.scriptContextRegistry = scriptContextRegistry; this.lastInlineCompileTime = System.nanoTime(); - this.setMaxCompilationsPerMinute(ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE.get(settings)); + this.setMaxCompilationsPerMinute( + ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE.get(settings)); } void registerClusterSettingsListeners(ClusterSettings clusterSettings) { - clusterSettings.addSettingsUpdateConsumer(ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE, this::setMaxCompilationsPerMinute); + clusterSettings.addSettingsUpdateConsumer(ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE, + this::setMaxCompilationsPerMinute); } void setMaxCompilationsPerMinute(Integer newMaxPerMinute) { this.totalCompilesPerMinute = newMaxPerMinute; // Reset the counter to allow new compilations this.scriptsPerMinCounter = totalCompilesPerMinute; - this.compilesAllowedPerNano = ((double) totalCompilesPerMinute) / TimeValue.timeValueMinutes(1).nanos(); + this.compilesAllowedPerNano = ((double) totalCompilesPerMinute) + / TimeValue.timeValueMinutes(1).nanos(); } /** - * Check whether there have been too many compilations within the last minute, throwing a circuit breaking exception if so. - * This is a variant of the token bucket algorithm: https://en.wikipedia.org/wiki/Token_bucket + * Check whether there have been too many compilations within the last minute, throwing a + * circuit breaking exception if so. This is a variant of the token bucket algorithm: + * https://en.wikipedia.org/wiki/Token_bucket * - * It can be thought of as a bucket with water, every time the bucket is checked, water is added proportional to the amount of time that - * elapsed since the last time it was checked. If there is enough water, some is removed and the request is allowed. If there is not - * enough water the request is denied. Just like a normal bucket, if water is added that overflows the bucket, the extra water/capacity - * is discarded - there can never be more water in the bucket than the size of the bucket. + * It can be thought of as a bucket with water, every time the bucket is checked, water is added + * proportional to the amount of time that elapsed since the last time it was checked. If there + * is enough water, some is removed and the request is allowed. If there is not enough water the + * request is denied. Just like a normal bucket, if water is added that overflows the bucket, + * the extra water/capacity is discarded - there can never be more water in the bucket than the + * size of the bucket. */ public void checkCompilationLimit() { long now = System.nanoTime(); @@ -84,16 +91,21 @@ public void checkCompilationLimit() { scriptsPerMinCounter -= 1.0; } else { // Otherwise reject the request - throw new CircuitBreakingException("[script] Too many dynamic script compilations within one minute, max: [" + - totalCompilesPerMinute + "/min]; please use on-disk, indexed, or scripts with parameters instead; " + - "this limit can be changed by the [" + ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE.getKey() + "] setting"); + throw new CircuitBreakingException( + "[script] Too many dynamic script compilations within one minute, max: [" + + totalCompilesPerMinute + "/min]; please use on-disk, indexed, or " + + "scripts with parameters instead; this limit can be changed by the [" + + ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE.getKey() + + "] setting"); } } - public boolean canExecuteScript(String lang, ScriptType scriptType, ScriptContext scriptContext) { + public boolean canExecuteScript(String lang, ScriptType scriptType, + ScriptContext scriptContext) { assert lang != null; if (scriptContextRegistry.isSupportedContext(scriptContext) == false) { - throw new IllegalArgumentException("script context [" + scriptContext.getKey() + "] not supported"); + throw new IllegalArgumentException( + "script context [" + scriptContext.getKey() + "] not supported"); } return scriptModes.getScriptEnabled(lang, scriptType, scriptContext); } diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index 25211911cfc7a..b638c05eb05c5 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -46,18 +46,21 @@ public interface Backend extends ScriptEngineService {} // NOCOMMIT this should private final ScriptPermits scriptPermits; private final CachingCompiler compiler; - public TemplateService(Settings settings, Environment env, ResourceWatcherService resourceWatcherService, - Backend backend, ScriptContextRegistry scriptContextRegistry, ScriptSettings scriptSettings, + public TemplateService(Settings settings, Environment env, + ResourceWatcherService resourceWatcherService, Backend backend, + ScriptContextRegistry scriptContextRegistry, ScriptSettings scriptSettings, ScriptMetrics scriptMetrics) throws IOException { Objects.requireNonNull(scriptContextRegistry); this.backend = backend; this.scriptPermits = new ScriptPermits(settings, scriptSettings, scriptContextRegistry); - this.compiler = new CachingCompiler(settings, scriptSettings, env, resourceWatcherService, scriptMetrics) { + this.compiler = new CachingCompiler(settings, scriptSettings, env, + resourceWatcherService, scriptMetrics) { @Override protected String cacheKeyForFile(String baseName, String extension) { if (false == backend.getType().equals(extension)) { - // For backwards compatibility templates are in the scripts directory and we must ignore all other templates + /* For backwards compatibility templates are in the scripts directory and we + * must ignore all other templates. */ return null; } return baseName; @@ -69,12 +72,13 @@ protected String cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { } @Override - protected StoredScriptSource lookupStoredScript(ClusterState clusterState, String cacheKey) { + protected StoredScriptSource lookupStoredScript(ClusterState clusterState, + String cacheKey) { ScriptMetaData scriptMetadata = clusterState.metaData().custom(ScriptMetaData.TYPE); if (scriptMetadata == null) { return null; } - + String id = cacheKey; // search template requests can possibly pass in the entire path instead // of just an id for looking up a stored script, so we parse the path and @@ -84,11 +88,15 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, Strin if (path.length == 3) { id = path[2]; - deprecationLogger.deprecated("use of [" + cacheKey + "] for looking up" + - " stored scripts/templates has been deprecated, use only [" + id + "] instead"); - } else if (path.length != 1) { - throw new IllegalArgumentException("illegal stored script format [" + id + "] use only "); - } + deprecationLogger + .deprecated("use of [" + cacheKey + "] for looking up" + + " stored scripts/templates has been deprecated, use only" + + " [" + id + "] instead"); + } else + if (path.length != 1) { + throw new IllegalArgumentException( + "illegal stored script format [" + id + "] use only "); + } return scriptMetadata.getStoredScript(id, "mustache"); } @@ -96,7 +104,8 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, Strin @Override protected boolean areAnyScriptContextsEnabled(String cacheKey, ScriptType scriptType) { for (ScriptContext scriptContext : scriptContextRegistry.scriptContexts()) { - if (scriptPermits.canExecuteScript(backend.getType(), scriptType, scriptContext)) { + if (scriptPermits.canExecuteScript(backend.getType(), scriptType, + scriptContext)) { return true; } } @@ -104,10 +113,13 @@ protected boolean areAnyScriptContextsEnabled(String cacheKey, ScriptType script } @Override - protected void checkCanExecuteScript(String cacheKey, ScriptType scriptType, ScriptContext scriptContext) { - if (scriptPermits.canExecuteScript(backend.getType(), scriptType, scriptContext) == false) { - throw new IllegalStateException("scripts of type [" + scriptType + "]," + - " operation [" + scriptContext.getKey() + "] and lang [" + backend.getType() + "] are disabled"); + protected void checkCanExecuteScript(String cacheKey, ScriptType scriptType, + ScriptContext scriptContext) { + if (scriptPermits.canExecuteScript(backend.getType(), scriptType, + scriptContext) == false) { + throw new IllegalStateException("scripts of type [" + scriptType + "]," + + " operation [" + scriptContext.getKey() + "] and lang [" + + backend.getType() + "] are disabled"); } } @@ -124,16 +136,20 @@ protected CompiledScript compile(ScriptType scriptType, String cacheKey) { @Override protected CompiledScript compileFileScript(String cacheKey, String body, Path file) { - Object compiled = backend.compile(file.getFileName().toString(), cacheKey, emptyMap()); + Object compiled = backend.compile(file.getFileName().toString(), cacheKey, + emptyMap()); return new CompiledScript(ScriptType.FILE, body, backend.getType(), compiled); } }; } - public BytesReference render(String idOrCode, ScriptType scriptType, ScriptContext scriptContext, - Map scriptParams) { - BytesReference b = (BytesReference) backend.executable(compiler.getScript(idOrCode, scriptType, scriptContext), scriptParams).run(); - ESLoggerFactory.getLogger(TemplateService.class).warn("ASDFASDF rendered [{}]", b.utf8ToString()); + public BytesReference render(String idOrCode, ScriptType scriptType, + ScriptContext scriptContext, Map scriptParams) { + BytesReference b = (BytesReference) backend + .executable(compiler.getScript(idOrCode, scriptType, scriptContext), scriptParams) + .run(); + ESLoggerFactory.getLogger(TemplateService.class).warn("ASDFASDF rendered [{}]", + b.utf8ToString()); return b; } diff --git a/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java b/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java index 601ef6f8de5d7..01ea513af02df 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java @@ -28,7 +28,8 @@ public class ScriptPermitsTests extends ESTestCase { public void testCompilationCircuitBreaking() throws Exception { ScriptContextRegistry contextRegistry = new ScriptContextRegistry(emptyList()); - ScriptSettings scriptSettings = new ScriptSettings(new ScriptEngineRegistry(emptyList()), null, contextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(new ScriptEngineRegistry(emptyList()), + null, contextRegistry); ScriptPermits permits = new ScriptPermits(Settings.EMPTY, scriptSettings, contextRegistry); permits.setMaxCompilationsPerMinute(1); permits.checkCompilationLimit(); // should pass diff --git a/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java b/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java index a0312f766c778..6407b9fbe7fb3 100644 --- a/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java +++ b/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java @@ -82,14 +82,16 @@ public void setup() throws IOException { scriptContextRegistry = new ScriptContextRegistry(contexts.values()); backend = mock(TemplateService.Backend.class); when(backend.getType()).thenReturn("test_template_backend"); - scriptSettings = new ScriptSettings(new ScriptEngineRegistry(emptyList()), backend, scriptContextRegistry); - scriptContexts = scriptContextRegistry.scriptContexts().toArray(new ScriptContext[scriptContextRegistry.scriptContexts().size()]); + scriptSettings = new ScriptSettings(new ScriptEngineRegistry(emptyList()), backend, + scriptContextRegistry); + scriptContexts = scriptContextRegistry.scriptContexts() + .toArray(new ScriptContext[scriptContextRegistry.scriptContexts().size()]); logger.info("--> setup script service"); scriptsFilePath = genericConfigFolder.resolve("scripts"); Files.createDirectories(scriptsFilePath); Environment environment = new Environment(settings); - templateService = new TemplateService(settings, environment, resourceWatcherService, backend, scriptContextRegistry, - scriptSettings, new ScriptMetrics()); + templateService = new TemplateService(settings, environment, resourceWatcherService, + backend, scriptContextRegistry, scriptSettings, new ScriptMetrics()); } public void testFoo() { diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java index 770007d6c77b4..d8c977c40bc79 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java @@ -22,10 +22,13 @@ import org.elasticsearch.action.Action; import org.elasticsearch.client.ElasticsearchClient; -public class DeleteStoredSearchTemplateAction - extends Action { +public class DeleteStoredSearchTemplateAction extends Action< + DeleteStoredSearchTemplateRequest, + DeleteStoredSearchTemplateResponse, + DeleteStoredSearchTemplateRequestBuilder> { - public static final DeleteStoredSearchTemplateAction INSTANCE = new DeleteStoredSearchTemplateAction(); + public static final DeleteStoredSearchTemplateAction INSTANCE = + new DeleteStoredSearchTemplateAction(); public static final String NAME = "cluster:admin/search/template/delete"; private DeleteStoredSearchTemplateAction() { diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java index d121f07b66e5c..0164eae7752fa 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java @@ -28,7 +28,9 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public class DeleteStoredSearchTemplateRequest extends AcknowledgedRequest { +public class DeleteStoredSearchTemplateRequest extends AcknowledgedRequest< + DeleteStoredSearchTemplateRequest> { + private String id; DeleteStoredSearchTemplateRequest() { diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequestBuilder.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequestBuilder.java index e0ab61e335ec9..ab0c563b60aee 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequestBuilder.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequestBuilder.java @@ -23,11 +23,12 @@ import org.elasticsearch.client.ElasticsearchClient; public class DeleteStoredSearchTemplateRequestBuilder extends AcknowledgedRequestBuilder< - DeleteStoredSearchTemplateRequest, - DeleteStoredSearchTemplateResponse, - DeleteStoredSearchTemplateRequestBuilder> { + DeleteStoredSearchTemplateRequest, + DeleteStoredSearchTemplateResponse, + DeleteStoredSearchTemplateRequestBuilder> { - public DeleteStoredSearchTemplateRequestBuilder(ElasticsearchClient client, DeleteStoredSearchTemplateAction action) { + public DeleteStoredSearchTemplateRequestBuilder(ElasticsearchClient client, + DeleteStoredSearchTemplateAction action) { super(client, action, new DeleteStoredSearchTemplateRequest()); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateAction.java index e8759a183b203..52ad8adcca410 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateAction.java @@ -22,10 +22,13 @@ import org.elasticsearch.action.Action; import org.elasticsearch.client.ElasticsearchClient; -public class GetStoredSearchTemplateAction extends Action { - public static final GetStoredSearchTemplateAction INSTANCE = new GetStoredSearchTemplateAction(); + public static final GetStoredSearchTemplateAction INSTANCE = + new GetStoredSearchTemplateAction(); public static final String NAME = "cluster:admin/search/template/get"; private GetStoredSearchTemplateAction() { diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java index 1b36cce049d12..a5a695c212b38 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java @@ -28,7 +28,9 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public class GetStoredSearchTemplateRequest extends MasterNodeReadRequest { +public class GetStoredSearchTemplateRequest extends MasterNodeReadRequest< + GetStoredSearchTemplateRequest> { + protected String id; GetStoredSearchTemplateRequest() { diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequestBuilder.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequestBuilder.java index 4e2df79948ea1..41f9929599987 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequestBuilder.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequestBuilder.java @@ -23,11 +23,12 @@ import org.elasticsearch.client.ElasticsearchClient; public class GetStoredSearchTemplateRequestBuilder extends MasterNodeReadOperationRequestBuilder< - GetStoredSearchTemplateRequest, - GetStoredSearchTemplateResponse, - GetStoredSearchTemplateRequestBuilder> { + GetStoredSearchTemplateRequest, + GetStoredSearchTemplateResponse, + GetStoredSearchTemplateRequestBuilder> { - public GetStoredSearchTemplateRequestBuilder(ElasticsearchClient client, GetStoredSearchTemplateAction action) { + public GetStoredSearchTemplateRequestBuilder(ElasticsearchClient client, + GetStoredSearchTemplateAction action) { super(client, action, new GetStoredSearchTemplateRequest()); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateAction.java index e67716f8e4480..f9b4c70e40da3 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateAction.java @@ -22,9 +22,13 @@ import org.elasticsearch.action.Action; import org.elasticsearch.client.ElasticsearchClient; -public class PutStoredSearchTemplateAction - extends Action { - public static final PutStoredSearchTemplateAction INSTANCE = new PutStoredSearchTemplateAction(); +public class PutStoredSearchTemplateAction extends Action< + PutStoredSearchTemplateRequest, + PutStoredSearchTemplateResponse, + PutStoredSearchTemplateRequestBuilder> { + + public static final PutStoredSearchTemplateAction INSTANCE = + new PutStoredSearchTemplateAction(); public static final String NAME = "cluster:admin/search/template/put"; private PutStoredSearchTemplateAction() { diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequest.java index 747ad9a98eabd..8f4904f922b91 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequest.java @@ -32,7 +32,8 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public class PutStoredSearchTemplateRequest extends AcknowledgedRequest { +public class PutStoredSearchTemplateRequest extends AcknowledgedRequest< + PutStoredSearchTemplateRequest> { private String id; private BytesReference content; private XContentType xContentType; @@ -40,7 +41,8 @@ public class PutStoredSearchTemplateRequest extends AcknowledgedRequest { - public PutStoredSearchTemplateRequestBuilder(ElasticsearchClient client, PutStoredSearchTemplateAction action) { + public PutStoredSearchTemplateRequestBuilder(ElasticsearchClient client, + PutStoredSearchTemplateAction action) { super(client, action, new PutStoredSearchTemplateRequest()); } @@ -38,7 +39,8 @@ public PutStoredSearchTemplateRequestBuilder setId(String id) { return this; } - public PutStoredSearchTemplateRequestBuilder setContent(BytesReference content, XContentType xContentType) { + public PutStoredSearchTemplateRequestBuilder setContent(BytesReference content, + XContentType xContentType) { request.content(content, xContentType); return this; } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredSearchTemplateAction.java index e897b221035b0..2d4926cee8a68 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredSearchTemplateAction.java @@ -38,10 +38,12 @@ public RestDeleteStoredSearchTemplateAction(Settings settings, RestController co } @Override - public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { + public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) + throws IOException { String id = request.param("id"); DeleteStoredSearchTemplateRequest deleteRequest = new DeleteStoredSearchTemplateRequest(id); - return channel -> client.execute(DeleteStoredSearchTemplateAction.INSTANCE, deleteRequest, new AcknowledgedRestListener<>(channel)); + return channel -> client.execute(DeleteStoredSearchTemplateAction.INSTANCE, deleteRequest, + new AcknowledgedRestListener<>(channel)); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredSearchTemplateAction.java index 76c71b6ec36ea..bb0e0b5352283 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredSearchTemplateAction.java @@ -49,7 +49,8 @@ public RestGetStoredSearchTemplateAction(Settings settings, RestController contr } @Override - public RestChannelConsumer prepareRequest(final RestRequest request, NodeClient client) throws IOException { + public RestChannelConsumer prepareRequest(final RestRequest request, NodeClient client) + throws IOException { String id = request.param("id"); GetStoredSearchTemplateRequest getRequest = new GetStoredSearchTemplateRequest(id); @@ -57,23 +58,28 @@ public RestChannelConsumer prepareRequest(final RestRequest request, NodeClient return channel -> client.execute(GetStoredSearchTemplateAction.INSTANCE, getRequest, new RestBuilderListener(channel) { @Override - public RestResponse buildResponse(GetStoredSearchTemplateResponse response, XContentBuilder builder) throws Exception { + public RestResponse buildResponse(GetStoredSearchTemplateResponse response, + XContentBuilder builder) throws Exception { builder.startObject(); - builder.field(_ID_PARSE_FIELD.getPreferredName(), id); + builder.field("_id", id); - builder.field(StoredScriptSource.LANG_PARSE_FIELD.getPreferredName(), Script.DEFAULT_TEMPLATE_LANG); + builder.field(StoredScriptSource.LANG_PARSE_FIELD.getPreferredName(), + Script.DEFAULT_TEMPLATE_LANG); StoredScriptSource source = response.getSource(); boolean found = source != null; - builder.field(FOUND_PARSE_FIELD.getPreferredName(), found); + builder.field("found", found); if (found) { - builder.field(StoredScriptSource.TEMPLATE_PARSE_FIELD.getPreferredName(), source.getCode()); + builder.field( + StoredScriptSource.TEMPLATE_PARSE_FIELD.getPreferredName(), + source.getCode()); } builder.endObject(); - return new BytesRestResponse(found ? RestStatus.OK : RestStatus.NOT_FOUND, builder); + return new BytesRestResponse(found ? RestStatus.OK : RestStatus.NOT_FOUND, + builder); } }); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredSearchTemplateAction.java index 84950d9cebc26..85a36bc4eafc1 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredSearchTemplateAction.java @@ -41,11 +41,14 @@ public RestPutStoredSearchTemplateAction(Settings settings, RestController contr } @Override - public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { + public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) + throws IOException { String id = request.param("id"); BytesReference content = request.content(); - PutStoredSearchTemplateRequest put = new PutStoredSearchTemplateRequest(id, content, request.getXContentType()); - return channel -> client.execute(PutStoredSearchTemplateAction.INSTANCE, put, new AcknowledgedRestListener<>(channel)); + PutStoredSearchTemplateRequest put = new PutStoredSearchTemplateRequest(id, content, + request.getXContentType()); + return channel -> client.execute(PutStoredSearchTemplateAction.INSTANCE, put, + new AcknowledgedRestListener<>(channel)); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredSearchTemplateAction.java index cd931b4cf2a9e..0a2d33c4c1f0b 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredSearchTemplateAction.java @@ -36,17 +36,19 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -public class TransportDeleteStoredSearchTemplateAction - extends TransportMasterNodeAction { +public class TransportDeleteStoredSearchTemplateAction extends TransportMasterNodeAction< + DeleteStoredSearchTemplateRequest, DeleteStoredSearchTemplateResponse> { private final TemplateService templateService; @Inject - public TransportDeleteStoredSearchTemplateAction(Settings settings, TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver, TemplateService templateService) { - super(settings, DeleteStoredSearchTemplateAction.NAME, transportService, clusterService, threadPool, actionFilters, - indexNameExpressionResolver, DeleteStoredSearchTemplateRequest::new); + public TransportDeleteStoredSearchTemplateAction(Settings settings, + TransportService transportService, ClusterService clusterService, ThreadPool threadPool, + ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, + TemplateService templateService) { + super(settings, DeleteStoredSearchTemplateAction.NAME, transportService, clusterService, + threadPool, actionFilters, indexNameExpressionResolver, + DeleteStoredSearchTemplateRequest::new); this.templateService = templateService; } @@ -62,26 +64,30 @@ protected DeleteStoredSearchTemplateResponse newResponse() { @Override protected void masterOperation(DeleteStoredSearchTemplateRequest request, ClusterState state, - ActionListener listener) throws Exception { + ActionListener listener) throws Exception { clusterService.submitStateUpdateTask("delete-search-template-" + request.id(), - new AckedClusterStateUpdateTask(request, listener) { + new AckedClusterStateUpdateTask(request, + listener) { @Override protected DeleteStoredSearchTemplateResponse newResponse(boolean acknowledged) { return new DeleteStoredSearchTemplateResponse(acknowledged); } - + @Override public ClusterState execute(ClusterState currentState) throws Exception { ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); - smd = ScriptMetaData.deleteStoredScript(smd, request.id(), templateService.getTemplateLanguage()); - MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); + smd = ScriptMetaData.deleteStoredScript(smd, request.id(), + templateService.getTemplateLanguage()); + MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()) + .putCustom(ScriptMetaData.TYPE, smd); return ClusterState.builder(currentState).metaData(mdb).build(); } }); } @Override - protected ClusterBlockException checkBlock(DeleteStoredSearchTemplateRequest request, ClusterState state) { + protected ClusterBlockException checkBlock(DeleteStoredSearchTemplateRequest request, + ClusterState state) { return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredSearchTemplateAction.java index cc02a0d1c0300..2a85033cb9560 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredSearchTemplateAction.java @@ -34,17 +34,19 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -public class TransportGetStoredSearchTemplateAction - extends TransportMasterNodeReadAction { +public class TransportGetStoredSearchTemplateAction extends TransportMasterNodeReadAction< + GetStoredSearchTemplateRequest, GetStoredSearchTemplateResponse> { private final TemplateService templateService; @Inject - public TransportGetStoredSearchTemplateAction(Settings settings, TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver, TemplateService templateService) { - super(settings, GetStoredSearchTemplateAction.NAME, transportService, clusterService, threadPool, actionFilters, - indexNameExpressionResolver, GetStoredSearchTemplateRequest::new); + public TransportGetStoredSearchTemplateAction(Settings settings, + TransportService transportService, ClusterService clusterService, ThreadPool threadPool, + ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, + TemplateService templateService) { + super(settings, GetStoredSearchTemplateAction.NAME, transportService, clusterService, + threadPool, actionFilters, indexNameExpressionResolver, + GetStoredSearchTemplateRequest::new); this.templateService = templateService; } @@ -60,19 +62,20 @@ protected GetStoredSearchTemplateResponse newResponse() { @Override protected void masterOperation(GetStoredSearchTemplateRequest request, ClusterState state, - ActionListener listener) throws Exception { + ActionListener listener) throws Exception { ScriptMetaData scriptMetadata = state.metaData().custom(ScriptMetaData.TYPE); if (scriptMetadata != null) { - listener.onResponse(new GetStoredSearchTemplateResponse( - scriptMetadata.getStoredScript(request.id(), templateService.getTemplateLanguage()))); + listener.onResponse(new GetStoredSearchTemplateResponse(scriptMetadata + .getStoredScript(request.id(), templateService.getTemplateLanguage()))); } else { listener.onResponse(null); } } @Override - protected ClusterBlockException checkBlock(GetStoredSearchTemplateRequest request, ClusterState state) { + protected ClusterBlockException checkBlock(GetStoredSearchTemplateRequest request, + ClusterState state) { return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java index 5cee30deb611b..befb03175ee16 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java @@ -39,18 +39,20 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -public class TransportPutStoredSearchTemplateAction - extends TransportMasterNodeAction { +public class TransportPutStoredSearchTemplateAction extends TransportMasterNodeAction< + PutStoredSearchTemplateRequest, PutStoredSearchTemplateResponse> { private final TemplateService templateService; private final int maxScriptSizeInBytes; @Inject - public TransportPutStoredSearchTemplateAction(Settings settings, TransportService transportService, ClusterService clusterService, - ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver, TemplateService templateService) { - super(settings, PutStoredSearchTemplateAction.NAME, transportService, clusterService, threadPool, actionFilters, - indexNameExpressionResolver, PutStoredSearchTemplateRequest::new); + public TransportPutStoredSearchTemplateAction(Settings settings, + TransportService transportService, ClusterService clusterService, ThreadPool threadPool, + ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, + TemplateService templateService) { + super(settings, PutStoredSearchTemplateAction.NAME, transportService, clusterService, + threadPool, actionFilters, indexNameExpressionResolver, + PutStoredSearchTemplateRequest::new); this.templateService = templateService; maxScriptSizeInBytes = ScriptService.SCRIPT_MAX_SIZE_IN_BYTES.get(settings); } @@ -67,43 +69,48 @@ protected PutStoredSearchTemplateResponse newResponse() { @Override protected void masterOperation(PutStoredSearchTemplateRequest request, ClusterState state, - ActionListener listener) throws Exception { + ActionListener listener) throws Exception { if (request.content().length() > maxScriptSizeInBytes) { - throw new IllegalArgumentException("exceeded max allowed stored script size in bytes [" + maxScriptSizeInBytes - + "] with size [" + request.content().length() + "] for script [" + request.id() + "]"); + throw new IllegalArgumentException("exceeded max allowed stored script size in bytes [" + + maxScriptSizeInBytes + "] with size [" + request.content().length() + + "] for script [" + request.id() + "]"); } - StoredScriptSource source = StoredScriptSource.parse(templateService.getTemplateLanguage(), request.content(), - request.xContentType()); + StoredScriptSource source = StoredScriptSource.parse(templateService.getTemplateLanguage(), + request.content(), request.xContentType()); try { templateService.checkCompileBeforeStore(source); } catch (IllegalArgumentException | ScriptException e) { - throw new IllegalArgumentException("failed to parse/compile stored search template [" + request.id() + "]" + - (source.getCode() == null ? "" : " using code [" + source.getCode() + "]"), e); + throw new IllegalArgumentException("failed to parse/compile stored search template [" + + request.id() + "]" + + (source.getCode() == null ? "" : " using code [" + source.getCode() + "]"), + e); } clusterService.submitStateUpdateTask("put-search-template-" + request.id(), - new AckedClusterStateUpdateTask(request, listener) { + new AckedClusterStateUpdateTask(request, + listener) { - @Override - protected PutStoredSearchTemplateResponse newResponse(boolean acknowledged) { - return new PutStoredSearchTemplateResponse(acknowledged); - } + @Override + protected PutStoredSearchTemplateResponse newResponse(boolean acknowledged) { + return new PutStoredSearchTemplateResponse(acknowledged); + } - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); - smd = ScriptMetaData.putStoredScript(smd, request.id(), source); - MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); + @Override + public ClusterState execute(ClusterState currentState) throws Exception { + ScriptMetaData smd = currentState.metaData().custom(ScriptMetaData.TYPE); + smd = ScriptMetaData.putStoredScript(smd, request.id(), source); + MetaData.Builder mdb = MetaData.builder(currentState.getMetaData()) + .putCustom(ScriptMetaData.TYPE, smd); - return ClusterState.builder(currentState).metaData(mdb).build(); - } - }); + return ClusterState.builder(currentState).metaData(mdb).build(); + } + }); } @Override - protected ClusterBlockException checkBlock(PutStoredSearchTemplateRequest request, ClusterState state) { + protected ClusterBlockException checkBlock(PutStoredSearchTemplateRequest request, + ClusterState state) { return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); } - } From e2e7f3d8bd82dc5b760ca06ac0d19d87c383ecc6 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 20 Mar 2017 12:00:19 -0400 Subject: [PATCH 06/37] Line length --- .../common/xcontent/XContentFactory.java | 3 --- .../stored/DeleteStoredSearchTemplateRequest.java | 11 +++++++---- .../stored/GetStoredSearchTemplateRequest.java | 11 +++++++---- 3 files changed, 14 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java b/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java index 4bb45055f5b85..0e6a13e60e022 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java @@ -277,9 +277,6 @@ public static XContentType xContentType(byte[] data, int offset, int length) { @Deprecated public static XContent xContent(BytesReference bytes) { XContentType type = xContentType(bytes); - if (type == null) { - throw new ElasticsearchParseException("Failed to derive xcontent for [" + bytes.utf8ToString() + "]"); // NOCOMMIT revert me. - } return xContent(type); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java index 0164eae7752fa..3a2d9e43fdb7a 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java @@ -50,10 +50,13 @@ public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; if (id == null || id.isEmpty()) { - validationException = addValidationError("must specify id for stored search template", validationException); - } else if (id.contains("#")) { - validationException = addValidationError("id cannot contain '#' for stored search template", validationException); - } + validationException = addValidationError("must specify id for stored search template", + validationException); + } else + if (id.contains("#")) { + validationException = addValidationError( + "id cannot contain '#' for stored search template", validationException); + } return validationException; } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java index a5a695c212b38..d96123cb6e937 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java @@ -50,10 +50,13 @@ public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; if (id == null || id.isEmpty()) { - validationException = addValidationError("must specify id for stored search template", validationException); - } else if (id.contains("#")) { - validationException = addValidationError("id cannot contain '#' for stored search template", validationException); - } + validationException = addValidationError("must specify id for stored search template", + validationException); + } else + if (id.contains("#")) { + validationException = addValidationError( + "id cannot contain '#' for stored search template", validationException); + } return validationException; } From 88be3a5e7472ff75a57808e66c5686266048cc2c Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 23 Mar 2017 13:24:55 -0400 Subject: [PATCH 07/37] Fix file templates --- .../elasticsearch/script/CachingCompiler.java | 12 +++++----- .../elasticsearch/script/ScriptService.java | 6 ++--- .../elasticsearch/script/TemplateService.java | 22 +++++++++---------- .../20_render_search_template.yaml | 7 +++--- 4 files changed, 23 insertions(+), 24 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index a1dadd09cd614..323ab87aaa9fe 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -76,13 +76,16 @@ public abstract class CachingCompiler implements ClusterStateListener private final ScriptMetrics scriptMetrics; + private final String type; + private volatile ClusterState clusterState; public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environment env, - ResourceWatcherService resourceWatcherService, ScriptMetrics scriptMetrics) + ResourceWatcherService resourceWatcherService, ScriptMetrics scriptMetrics, String type) throws IOException { int cacheMaxSize = ScriptService.SCRIPT_CACHE_SIZE_SETTING.get(settings); this.scriptMetrics = scriptMetrics; + this.type = type; CacheBuilder cacheBuilder = CacheBuilder.builder(); if (cacheMaxSize >= 0) { @@ -149,8 +152,6 @@ protected abstract void checkCanExecuteScript(CacheKeyT cacheKey, ScriptType scr public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, ScriptContext scriptContext) { - ESLoggerFactory.getLogger(QueryRewriteContext.class).warn("ASDFASDF get {} {}", cacheKey, - scriptType); Objects.requireNonNull(cacheKey); // First resolve stored scripts so so we have accurate parameters for checkCanExecuteScript @@ -165,10 +166,9 @@ public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, if (scriptType == ScriptType.FILE) { CompiledScript compiled = fileScripts.get(cacheKey); if (compiled == null) { - throw new IllegalArgumentException("unable to find file script " + cacheKey); + throw new IllegalArgumentException("unable to find file " + type + + " [" + cacheKey + "]"); } - ESLoggerFactory.getLogger(QueryRewriteContext.class).warn("ASDFASDF got file {}", - compiled); return compiled; } diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index c7d8246efaac4..d2d233af8ccc4 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -104,7 +104,7 @@ public ScriptService(Settings settings, Environment env, ResourceWatcherService maxScriptSizeInBytes = ScriptService.SCRIPT_MAX_SIZE_IN_BYTES.get(settings); this.scriptMetrics = scriptMetrics; this.scriptPermits = new ScriptPermits(settings, scriptSettings, scriptContextRegistry); - this.compiler = new CachingCompiler(settings, scriptSettings, env, resourceWatcherService, scriptMetrics) { + this.compiler = new CachingCompiler(settings, scriptSettings, env, resourceWatcherService, scriptMetrics, "script") { @Override protected CacheKey cacheKeyForFile(String baseName, String extension) { if (extension.equals("mustache")) { @@ -388,11 +388,11 @@ public int hashCode() { @Override public String toString() { - String result = "[lang=" + lang + ", id=" + idOrCode; + String result = "lang=" + lang + ", id=" + idOrCode; if (options != null) { result += ", options " + options; } - return result + "]"; + return result; } } } diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index b638c05eb05c5..f9288be8be8a6 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -55,12 +55,12 @@ public TemplateService(Settings settings, Environment env, this.backend = backend; this.scriptPermits = new ScriptPermits(settings, scriptSettings, scriptContextRegistry); this.compiler = new CachingCompiler(settings, scriptSettings, env, - resourceWatcherService, scriptMetrics) { + resourceWatcherService, scriptMetrics, "template") { @Override protected String cacheKeyForFile(String baseName, String extension) { if (false == backend.getType().equals(extension)) { /* For backwards compatibility templates are in the scripts directory and we - * must ignore all other templates. */ + * must ignore everything but templates. */ return null; } return baseName; @@ -88,15 +88,13 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, if (path.length == 3) { id = path[2]; - deprecationLogger - .deprecated("use of [" + cacheKey + "] for looking up" - + " stored scripts/templates has been deprecated, use only" - + " [" + id + "] instead"); - } else - if (path.length != 1) { - throw new IllegalArgumentException( - "illegal stored script format [" + id + "] use only "); - } + deprecationLogger.deprecated("use of [{}] for looking up stored " + + "scripts/templates has been deprecated, use only [{}] instead", + cacheKey, id); + } else if (path.length != 1) { + throw new IllegalArgumentException( "illegal stored script format [" + id + + "] use only "); + } return scriptMetadata.getStoredScript(id, "mustache"); } @@ -136,7 +134,7 @@ protected CompiledScript compile(ScriptType scriptType, String cacheKey) { @Override protected CompiledScript compileFileScript(String cacheKey, String body, Path file) { - Object compiled = backend.compile(file.getFileName().toString(), cacheKey, + Object compiled = backend.compile(file.getFileName().toString(), body, emptyMap()); return new CompiledScript(ScriptType.FILE, body, backend.getType(), compiled); } diff --git a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/20_render_search_template.yaml b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/20_render_search_template.yaml index f6fc458a08ec2..8243c1a568873 100644 --- a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/20_render_search_template.yaml +++ b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/20_render_search_template.yaml @@ -149,8 +149,9 @@ body: { "id" : "1", "params" : { "my_value" : "value1_foo", "my_size" : 1 } } - match: { hits.total: 1 } +--- +"Missing file template": - do: - catch: /unable.to.find.file.script.\[simple1\].using.lang.\[mustache\]/ + catch: /unable.to.find.file.template.\[missing\]/ search_template: - body: { "file" : "simple1"} - + body: { "file" : "missing"} From 7df0a1693b7c43205902f51c14c43f85f360c646 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 23 Mar 2017 17:06:54 -0400 Subject: [PATCH 08/37] More tests --- .../resources/checkstyle_suppressions.xml | 1 - .../org/elasticsearch/index/IndexService.java | 7 + .../index/query/QueryRewriteContext.java | 13 +- .../index/query/QueryShardContext.java | 9 +- .../elasticsearch/ingest/IngestService.java | 7 +- .../ingest/InternalTemplateService.java | 23 +--- .../java/org/elasticsearch/node/Node.java | 6 +- .../elasticsearch/script/CachingCompiler.java | 6 +- .../elasticsearch/script/ScriptService.java | 22 +++- .../elasticsearch/script/TemplateService.java | 41 ++++-- .../suggest/phrase/PhraseSuggester.java | 9 +- .../phrase/PhraseSuggestionBuilder.java | 8 +- .../phrase/PhraseSuggestionContext.java | 13 +- .../ingest/IngestServiceTests.java | 12 +- .../script/TemplateServiceTests.java | 122 ++++++++++++++++-- .../search/suggest/SuggestSearchIT.java | 8 +- .../TransportSearchTemplateAction.java | 12 +- .../ingest/AbstractScriptTestCase.java | 20 +-- .../10_pipeline_with_mustache_templates.yaml | 2 +- 19 files changed, 235 insertions(+), 106 deletions(-) diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index ab0a75a007aa4..a52b2c9039670 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -2988,7 +2988,6 @@ - diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index c64311af5a054..7420ced295634 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -499,6 +499,13 @@ public ScriptService getScriptService() { return scriptService; } + /** + * The {@link TemplateService} to use for this index. + */ + public TemplateService getTemplateService() { + return null; + } + List getIndexOperationListeners() { // pkg private for testing return indexingOperationListeners; } diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java index 27d5c294fe5be..e5177e7e368ac 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java @@ -21,12 +21,10 @@ import org.apache.lucene.index.IndexReader; import org.elasticsearch.client.Client; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptService; @@ -107,9 +105,14 @@ public long nowInMillis() { return nowInMillis.getAsLong(); } - public BytesReference getTemplateBytes(Script template) { - return templateService.render(template.getIdOrCode(), template.getType(), ScriptContext.Standard.SEARCH, - template.getParams()); + public BytesReference getTemplateBytes(Script template) { // TODO remove this and use getTemplateService directly + return templateService + .template(template.getIdOrCode(), template.getType(), ScriptContext.Standard.SEARCH) + .apply(template.getParams()); + } + + public ScriptService getScriptService() { + return scriptService; } public TemplateService getTemplateService() { diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index 76a4d9a549d1f..0b08ccbd8ae2f 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -356,13 +356,14 @@ public final ExecutableScript getExecutableScript(Script script, ScriptContext c } /** - * Returns a lazily created {@link ExecutableScript} that is compiled immediately but can be pulled later once all - * parameters are available. + * Returns {@link Function} representing a script that is compiled immediately but can be pulled + * later once all parameters are available. */ - public final Function, ExecutableScript> getLazyExecutableScript(Script script, ScriptContext context) { + public final Function, ExecutableScript> getLazyExecutableScript( + Script script, ScriptContext context) { failIfFrozen(); CompiledScript executable = scriptService.compile(script, context); - return (p) -> scriptService.executable(executable, p); + return (p) -> scriptService.executable(executable, p); } /** diff --git a/core/src/main/java/org/elasticsearch/ingest/IngestService.java b/core/src/main/java/org/elasticsearch/ingest/IngestService.java index 5249ed7a7dc84..8888ccf7cc350 100644 --- a/core/src/main/java/org/elasticsearch/ingest/IngestService.java +++ b/core/src/main/java/org/elasticsearch/ingest/IngestService.java @@ -40,11 +40,12 @@ public class IngestService { private final PipelineStore pipelineStore; private final PipelineExecutionService pipelineExecutionService; - public IngestService(Settings settings, ThreadPool threadPool, - Environment env, ScriptService scriptService, AnalysisRegistry analysisRegistry, + public IngestService(Settings settings, ThreadPool threadPool, Environment env, + org.elasticsearch.script.TemplateService esTemplateService, + ScriptService scriptService, AnalysisRegistry analysisRegistry, List ingestPlugins) { - final TemplateService templateService = new InternalTemplateService(scriptService); + final TemplateService templateService = new InternalTemplateService(esTemplateService); Processor.Parameters parameters = new Processor.Parameters(env, scriptService, templateService, analysisRegistry, threadPool.getThreadContext()); Map processorFactories = new HashMap<>(); diff --git a/core/src/main/java/org/elasticsearch/ingest/InternalTemplateService.java b/core/src/main/java/org/elasticsearch/ingest/InternalTemplateService.java index 26d6737706bcb..953212c4b9bea 100644 --- a/core/src/main/java/org/elasticsearch/ingest/InternalTemplateService.java +++ b/core/src/main/java/org/elasticsearch/ingest/InternalTemplateService.java @@ -20,22 +20,18 @@ package org.elasticsearch.ingest; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.script.CompiledScript; -import org.elasticsearch.script.ExecutableScript; -import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptContext; -import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptType; -import java.util.Collections; import java.util.Map; +import java.util.function.Function; public class InternalTemplateService implements TemplateService { - private final ScriptService scriptService; + private final org.elasticsearch.script.TemplateService templateService; - InternalTemplateService(ScriptService scriptService) { - this.scriptService = scriptService; + InternalTemplateService(org.elasticsearch.script.TemplateService templateService) { + this.templateService = templateService; } @Override @@ -43,17 +39,12 @@ public Template compile(String template) { int mustacheStart = template.indexOf("{{"); int mustacheEnd = template.indexOf("}}"); if (mustacheStart != -1 && mustacheEnd != -1 && mustacheStart < mustacheEnd) { - Script script = new Script(ScriptType.INLINE, "mustache", template, Collections.emptyMap()); - CompiledScript compiledScript = scriptService.compile(script, ScriptContext.Standard.INGEST); + Function, BytesReference> compiled = templateService.template( + template, ScriptType.INLINE, ScriptContext.Standard.INGEST); return new Template() { @Override public String execute(Map model) { - ExecutableScript executableScript = scriptService.executable(compiledScript, model); - Object result = executableScript.run(); - if (result instanceof BytesReference) { - return ((BytesReference) result).utf8ToString(); - } - return String.valueOf(result); + return compiled.apply(model).utf8ToString(); } @Override diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index bca6ef9685c0a..90741ae06ebfb 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -335,8 +335,10 @@ protected Node(final Environment environment, Collection clusterService.addListener(scriptModule.getScriptService()); // NOCOMMIT why not do this in the service's ctor? clusterService.addListener(scriptModule.getTemplateService()); resourcesToClose.add(clusterService); - final IngestService ingestService = new IngestService(settings, threadPool, this.environment, - scriptModule.getScriptService(), analysisModule.getAnalysisRegistry(), pluginsService.filterPlugins(IngestPlugin.class)); + final IngestService ingestService = new IngestService(settings, threadPool, + this.environment, scriptModule.getTemplateService(), + scriptModule.getScriptService(), analysisModule.getAnalysisRegistry(), + pluginsService.filterPlugins(IngestPlugin.class)); final ClusterInfoService clusterInfoService = newClusterInfoService(settings, clusterService, threadPool, client); ModulesBuilder modules = new ModulesBuilder(); diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 323ab87aaa9fe..3cf190e4c75a1 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -101,7 +101,7 @@ public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environ cacheExpire); this.cache = cacheBuilder.removalListener(new CacheRemovalListener()).build(); - // add file watcher for file scripts + // add file watcher for file scripts and templates scriptsDirectory = env.scriptsFile(); if (logger.isTraceEnabled()) { logger.trace("Using scripts directory [{}] ", scriptsDirectory); @@ -198,7 +198,7 @@ public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, // TODO: remove this try-catch when all script engines have good exceptions! throw good; // its already good } catch (Exception exception) { - throw new GeneralScriptException("Failed to compile " + cacheKey, exception); + throw new GeneralScriptException("Failed to compile [" + cacheKey + "]", exception); } scriptMetrics.onCompilation(); cache.put(cacheKey, compiledScript); @@ -211,7 +211,7 @@ private CacheKeyT getScriptFromClusterState(CacheKeyT cacheKey) { if (source == null) { throw new ResourceNotFoundException( - "unable to find script [" + cacheKey + "] in cluster state"); + "unable to find " + type + " [" + cacheKey + "] in cluster state"); } return cacheKeyFromClusterState(source); } diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index d2d233af8ccc4..c87f068caea37 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -51,6 +51,7 @@ import java.util.Map; import java.util.Objects; +import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; public class ScriptService extends AbstractComponent implements Closeable, ClusterStateListener { @@ -126,8 +127,15 @@ protected CacheKey cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { @Override protected StoredScriptSource lookupStoredScript(ClusterState clusterState, CacheKey cacheKey) { - ScriptMetaData scriptMetadata = clusterState.metaData().custom(ScriptMetaData.TYPE); - if (scriptMetadata == null) { + if (clusterState == null) { + return null; + } + MetaData metaData = clusterState.metaData(); + if (metaData == null) { + return null; + } + ScriptMetaData scriptMetaData = clusterState.metaData().custom(ScriptMetaData.TYPE); + if (scriptMetaData == null) { return null; } @@ -155,7 +163,7 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, Cache throw new IllegalArgumentException("illegal stored script format [" + id + "] use only "); } - return scriptMetadata.getStoredScript(id, cacheKey.lang); + return scriptMetaData.getStoredScript(id, cacheKey.lang); } @Override @@ -362,7 +370,7 @@ private static final class CacheKey { private CacheKey(String lang, String idOrCode, Map options) { this.lang = lang; this.idOrCode = idOrCode; - this.options = options; + this.options = options == null ? emptyMap() : options; } @Override @@ -374,7 +382,7 @@ public boolean equals(Object o) { if (lang != null ? !lang.equals(cacheKey.lang) : cacheKey.lang != null) return false; if (!idOrCode.equals(cacheKey.idOrCode)) return false; - return options != null ? options.equals(cacheKey.options) : cacheKey.options == null; + return options.equals(cacheKey.options); } @@ -382,14 +390,14 @@ public boolean equals(Object o) { public int hashCode() { int result = lang != null ? lang.hashCode() : 0; result = 31 * result + idOrCode.hashCode(); - result = 31 * result + (options != null ? options.hashCode() : 0); + result = 31 * result + options.hashCode(); return result; } @Override public String toString() { String result = "lang=" + lang + ", id=" + idOrCode; - if (options != null) { + if (false == options.isEmpty()) { result += ", options " + options; } return result; diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index f9288be8be8a6..d072872327a36 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -23,6 +23,7 @@ import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.ESLoggerFactory; @@ -34,11 +35,12 @@ import java.nio.file.Path; import java.util.Map; import java.util.Objects; +import java.util.function.Function; import static java.util.Collections.emptyMap; public class TemplateService implements ClusterStateListener { - public interface Backend extends ScriptEngineService {} // NOCOMMIT this should diverge.... + public interface Backend extends ScriptEngineService {} // TODO customize this for templates private static final Logger logger = ESLoggerFactory.getLogger(TemplateService.class); private static final DeprecationLogger deprecationLogger = new DeprecationLogger(logger); @@ -74,8 +76,15 @@ protected String cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { @Override protected StoredScriptSource lookupStoredScript(ClusterState clusterState, String cacheKey) { - ScriptMetaData scriptMetadata = clusterState.metaData().custom(ScriptMetaData.TYPE); - if (scriptMetadata == null) { + if (clusterState == null) { + return null; + } + MetaData metaData = clusterState.metaData(); + if (metaData == null) { + return null; + } + ScriptMetaData scriptMetaData = clusterState.metaData().custom(ScriptMetaData.TYPE); + if (scriptMetaData == null) { return null; } @@ -96,7 +105,7 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, + "] use only "); } - return scriptMetadata.getStoredScript(id, "mustache"); + return scriptMetaData.getStoredScript(id, backend.getExtension()); } @Override @@ -141,14 +150,22 @@ protected CompiledScript compileFileScript(String cacheKey, String body, Path fi }; } - public BytesReference render(String idOrCode, ScriptType scriptType, - ScriptContext scriptContext, Map scriptParams) { - BytesReference b = (BytesReference) backend - .executable(compiler.getScript(idOrCode, scriptType, scriptContext), scriptParams) - .run(); - ESLoggerFactory.getLogger(TemplateService.class).warn("ASDFASDF rendered [{}]", - b.utf8ToString()); - return b; + /** + * Lookup and/or compile a template. + * + * @param idOrCode template to look up and/or compile + * @param type whether to compile ({link ScriptType#INLINE}), lookup from cluster state + * ({@link ScriptType#STORED}), or lookup from disk ({@link ScriptType#FILE}) + * @param context context in which the template is being run + * @return the template + */ + public Function, BytesReference> template(String idOrCode, + ScriptType type, ScriptContext context) { + CompiledScript compiled = compiler.getScript(idOrCode, type, context); + return params -> { + ExecutableScript executable = backend.executable(compiled, params); + return (BytesReference) executable.run(); + }; } /** diff --git a/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java b/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java index fc862f6363864..6cdc22c5fd2f0 100644 --- a/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java +++ b/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java @@ -103,20 +103,19 @@ public Suggestion> innerExecute(String name, P response.addTerm(resultEntry); final BytesRefBuilder byteSpare = new BytesRefBuilder(); - final Function, ExecutableScript> collateScript = suggestion.getCollateQueryScript(); - final boolean collatePrune = (collateScript != null) && suggestion.collatePrune(); + final boolean collatePrune = + suggestion.getCollateQuery() != null && suggestion.collatePrune(); for (int i = 0; i < checkerResult.corrections.length; i++) { Correction correction = checkerResult.corrections[i]; spare.copyUTF8Bytes(correction.join(SEPARATOR, byteSpare, null, null)); boolean collateMatch = true; - if (collateScript != null) { + if (suggestion.getCollateQuery() != null) { // Checks if the template query collateScript yields any documents // from the index for a correction, collateMatch is updated final Map vars = suggestion.getCollateScriptParams(); vars.put(SUGGESTION_TEMPLATE_VAR_NAME, spare.toString()); QueryShardContext shardContext = suggestion.getShardContext(); - final ExecutableScript executable = collateScript.apply(vars); - final BytesReference querySource = (BytesReference) executable.run(); + final BytesReference querySource = suggestion.getCollateQuery().apply(vars); try (XContentParser parser = XContentFactory.xContent(querySource).createParser(shardContext.getXContentRegistry(), querySource)) { QueryBuilder innerQueryBuilder = shardContext.newParseContext(parser).parseInnerQueryBuilder(); diff --git a/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java b/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java index 49c5f5a9fd75f..137dc24b336d0 100644 --- a/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java @@ -37,7 +37,6 @@ import org.elasticsearch.index.analysis.TokenFilterFactory; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.query.QueryShardContext; -import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptType; @@ -54,7 +53,6 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Set; -import java.util.function.Function; /** * Defines the actual suggest command for phrase suggestions ( phrase). @@ -630,9 +628,9 @@ public SuggestionContext build(QueryShardContext context) throws IOException { } if (this.collateQuery != null) { - Function, ExecutableScript> compiledScript = context.getLazyExecutableScript(this.collateQuery, - ScriptContext.Standard.SEARCH); - suggestionContext.setCollateQueryScript(compiledScript); + suggestionContext.setCollateQuery( + context.getTemplateService().template(collateQuery.getIdOrCode(), + collateQuery.getType(), ScriptContext.Standard.SEARCH)); if (this.collateParams != null) { suggestionContext.setCollateScriptParams(this.collateParams); } diff --git a/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionContext.java b/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionContext.java index 99e2e18496b61..52338ed75a4dd 100644 --- a/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionContext.java +++ b/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionContext.java @@ -22,9 +22,8 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.Terms; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.index.query.QueryShardContext; -import org.elasticsearch.script.CompiledScript; -import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.search.suggest.DirectSpellcheckerSettings; import org.elasticsearch.search.suggest.SuggestionSearchContext.SuggestionContext; @@ -54,7 +53,7 @@ class PhraseSuggestionContext extends SuggestionContext { private boolean requireUnigram = DEFAULT_REQUIRE_UNIGRAM; private BytesRef preTag; private BytesRef postTag; - private Function, ExecutableScript> collateQueryScript; + private Function, BytesReference> collateQuery; private boolean prune = DEFAULT_COLLATE_PRUNE; private List generators = new ArrayList<>(); private Map collateScriptParams = new HashMap<>(1); @@ -194,12 +193,12 @@ public BytesRef getPostTag() { return postTag; } - Function, ExecutableScript> getCollateQueryScript() { - return collateQueryScript; + Function, BytesReference> getCollateQuery() { + return collateQuery; } - void setCollateQueryScript( Function, ExecutableScript> collateQueryScript) { - this.collateQueryScript = collateQueryScript; + void setCollateQuery(Function, BytesReference> collateQuery) { + this.collateQuery = collateQuery; } Map getCollateScriptParams() { diff --git a/core/src/test/java/org/elasticsearch/ingest/IngestServiceTests.java b/core/src/test/java/org/elasticsearch/ingest/IngestServiceTests.java index 3a842a4690afa..6d5eea97207fa 100644 --- a/core/src/test/java/org/elasticsearch/ingest/IngestServiceTests.java +++ b/core/src/test/java/org/elasticsearch/ingest/IngestServiceTests.java @@ -30,7 +30,7 @@ import org.mockito.Mockito; public class IngestServiceTests extends ESTestCase { - private final IngestPlugin DUMMY_PLUGIN = new IngestPlugin() { + private static final IngestPlugin DUMMY_PLUGIN = new IngestPlugin() { @Override public Map getProcessors(Processor.Parameters parameters) { return Collections.singletonMap("foo", (factories, tag, config) -> null); @@ -39,8 +39,10 @@ public Map getProcessors(Processor.Parameters paramet public void testIngestPlugin() { ThreadPool tp = Mockito.mock(ThreadPool.class); - IngestService ingestService = new IngestService(Settings.EMPTY, tp, null, null, null, Collections.singletonList(DUMMY_PLUGIN)); - Map factories = ingestService.getPipelineStore().getProcessorFactories(); + IngestService ingestService = new IngestService(Settings.EMPTY, tp, null, null, null, null, + Collections.singletonList(DUMMY_PLUGIN)); + Map factories = ingestService.getPipelineStore() + .getProcessorFactories(); assertTrue(factories.containsKey("foo")); assertEquals(1, factories.size()); } @@ -48,8 +50,8 @@ public void testIngestPlugin() { public void testIngestPluginDuplicate() { ThreadPool tp = Mockito.mock(ThreadPool.class); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> - new IngestService(Settings.EMPTY, tp, null, null, null, Arrays.asList(DUMMY_PLUGIN, DUMMY_PLUGIN)) - ); + new IngestService(Settings.EMPTY, tp, null, null, null, null, + Arrays.asList(DUMMY_PLUGIN, DUMMY_PLUGIN))); assertTrue(e.getMessage(), e.getMessage().contains("already registered")); } } diff --git a/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java b/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java index 6407b9fbe7fb3..7f12b24a14310 100644 --- a/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java +++ b/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java @@ -19,8 +19,16 @@ package org.elasticsearch.script; +import org.elasticsearch.ResourceNotFoundException; +import org.elasticsearch.cluster.ClusterChangedEvent; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; +import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.watcher.ResourceWatcherService; import org.junit.Before; @@ -33,16 +41,12 @@ import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; -import static java.util.Collections.singletonMap; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class TemplateServiceTests extends ESTestCase { private ResourceWatcherService resourceWatcherService; private ScriptContextRegistry scriptContextRegistry; private ScriptSettings scriptSettings; - private ScriptContext[] scriptContexts; - private Path scriptsFilePath; + private Path fileTemplatesPath; private Settings settings; private TemplateService.Backend backend; private TemplateService templateService; @@ -80,21 +84,113 @@ public void setup() throws IOException { contexts.put(context, new ScriptContext.Plugin(plugin, operation)); } scriptContextRegistry = new ScriptContextRegistry(contexts.values()); - backend = mock(TemplateService.Backend.class); - when(backend.getType()).thenReturn("test_template_backend"); + backend = new DummyBackend(); scriptSettings = new ScriptSettings(new ScriptEngineRegistry(emptyList()), backend, scriptContextRegistry); - scriptContexts = scriptContextRegistry.scriptContexts() - .toArray(new ScriptContext[scriptContextRegistry.scriptContexts().size()]); logger.info("--> setup script service"); - scriptsFilePath = genericConfigFolder.resolve("scripts"); - Files.createDirectories(scriptsFilePath); + fileTemplatesPath = genericConfigFolder.resolve("scripts"); + Files.createDirectories(fileTemplatesPath); Environment environment = new Environment(settings); templateService = new TemplateService(settings, environment, resourceWatcherService, backend, scriptContextRegistry, scriptSettings, new ScriptMetrics()); } - public void testFoo() { - // NOCOMMIT something here + public void testFileTemplates() throws IOException { + String body = "{\"test\":\"test\"}"; + Path testFileWithExt = fileTemplatesPath.resolve("test.test_template_backend"); + Streams.copy(body.getBytes("UTF-8"), Files.newOutputStream(testFileWithExt)); + resourceWatcherService.notifyNow(); + + assertEquals(new BytesArray(body), templateService.template("test", ScriptType.FILE, + ScriptContext.Standard.SEARCH).apply(emptyMap())); + + Files.delete(testFileWithExt); + resourceWatcherService.notifyNow(); + + Exception e = expectThrows(IllegalArgumentException.class, () -> templateService + .template("test", ScriptType.FILE, ScriptContext.Standard.SEARCH)); + assertEquals("unable to find file template [test]", e.getMessage()); + } + + public void testStoredTemplates() throws IOException { + String body = "{\"test\":\"test\"}"; + + ClusterState newState = ClusterState.builder(new ClusterName("test")) + .metaData(MetaData.builder() + .putCustom(ScriptMetaData.TYPE, new ScriptMetaData.Builder(null) + .storeScript("test_template_backend#test", new StoredScriptSource( + "test_template_backend", body, emptyMap())) + .build()) + .build()) + .build(); + templateService.clusterChanged( + new ClusterChangedEvent("test", newState, ClusterState.EMPTY_STATE)); + + assertEquals(new BytesArray(body), templateService.template("test", ScriptType.STORED, + ScriptContext.Standard.SEARCH).apply(emptyMap())); + + ClusterState oldState = newState; + newState = ClusterState.builder(oldState) + .metaData(MetaData.builder(oldState.metaData()) + .putCustom(ScriptMetaData.TYPE, new ScriptMetaData.Builder(null).build()) + .build()) + .build(); + templateService.clusterChanged(new ClusterChangedEvent("test", newState, oldState)); + + Exception e = expectThrows(ResourceNotFoundException.class, () -> templateService + .template("test", ScriptType.STORED, ScriptContext.Standard.SEARCH)); + assertEquals("unable to find template [test] in cluster state", e.getMessage()); + } + + public void testInlineTemplates() throws IOException { + String body = "{\"test\":\"test\"}"; + + assertEquals(new BytesArray(body), templateService.template(body, ScriptType.INLINE, + ScriptContext.Standard.SEARCH).apply(emptyMap())); + } + + /** + * Dummy backend that just returns the script's source when run. + */ + private static class DummyBackend implements TemplateService.Backend { + @Override + public String getType() { + return "test_template_backend"; + } + + @Override + public String getExtension() { + return "test_template_backend"; + } + + @Override + public Object compile(String scriptName, String scriptSource, Map params) { + return scriptSource; + } + + @Override + public ExecutableScript executable(CompiledScript compiledScript, + Map vars) { + return new ExecutableScript() { + @Override + public void setNextVar(String name, Object value) { + } + + @Override + public Object run() { + return new BytesArray((String) compiledScript.compiled()); + } + }; + } + + @Override + public SearchScript search(CompiledScript compiledScript, SearchLookup lookup, + Map vars) { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException { + } } } diff --git a/core/src/test/java/org/elasticsearch/search/suggest/SuggestSearchIT.java b/core/src/test/java/org/elasticsearch/search/suggest/SuggestSearchIT.java index 82f06441ef6da..d4e88283c59d2 100644 --- a/core/src/test/java/org/elasticsearch/search/suggest/SuggestSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/suggest/SuggestSearchIT.java @@ -22,7 +22,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.search.ReduceSearchPhaseException; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; @@ -34,8 +33,9 @@ import org.elasticsearch.plugins.ScriptPlugin; import org.elasticsearch.script.CompiledScript; import org.elasticsearch.script.ExecutableScript; -import org.elasticsearch.script.ScriptEngineService; import org.elasticsearch.script.SearchScript; +import org.elasticsearch.script.TemplateService; +import org.elasticsearch.script.TemplateService.Backend; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.suggest.phrase.DirectCandidateGeneratorBuilder; import org.elasticsearch.search.suggest.phrase.Laplace; @@ -1112,12 +1112,12 @@ protected Collection> nodePlugins() { public static class DummyTemplatePlugin extends Plugin implements ScriptPlugin { @Override - public ScriptEngineService getScriptEngineService(Settings settings) { + public Backend getTemplateBackend() { return new DummyTemplateScriptEngine(); } } - public static class DummyTemplateScriptEngine implements ScriptEngineService { + public static class DummyTemplateScriptEngine implements TemplateService.Backend { // The collate query setter is hard coded to use mustache, so lets lie in this test about the script plugin, // which makes the collate code thinks mustache is evaluating the query. diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java index 1a58093dfa1c3..e5efd073ec181 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java @@ -34,14 +34,15 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.query.QueryParseContext; -import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.TemplateService; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import java.util.Collections; +import java.util.Map; +import java.util.function.Function; +import static java.util.Collections.emptyMap; import static org.elasticsearch.script.ScriptContext.Standard.SEARCH; public class TransportSearchTemplateAction extends HandledTransportAction { @@ -65,8 +66,11 @@ public TransportSearchTemplateAction(Settings settings, ThreadPool threadPool, T protected void doExecute(SearchTemplateRequest request, ActionListener listener) { final SearchTemplateResponse response = new SearchTemplateResponse(); try { - BytesReference source = templateService.render(request.getScript(), request.getScriptType(), SEARCH, - request.getScriptParams() == null ? Collections.emptyMap() : request.getScriptParams()); + Function, BytesReference> template = + templateService.template(request.getScript(), request.getScriptType(), SEARCH); + Map params = + request.getScriptParams() == null ? emptyMap() : request.getScriptParams(); + BytesReference source = template.apply(params); response.setSource(source); if (request.isSimulate()) { diff --git a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java b/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java index 654b27c4dae82..53a131b5a1ec4 100644 --- a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java +++ b/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/AbstractScriptTestCase.java @@ -30,8 +30,7 @@ import org.elasticsearch.test.ESTestCase; import org.junit.Before; -import java.util.Arrays; -import java.util.Collections; +import static java.util.Collections.emptyList; public abstract class AbstractScriptTestCase extends ESTestCase { @@ -43,13 +42,16 @@ public void init() throws Exception { .put("path.home", createTempDir()) .put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), false) .build(); - ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Arrays.asList(new MustacheScriptEngineService())); - ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); - - ScriptService scriptService = new ScriptService(settings, new Environment(settings), null, - scriptEngineRegistry, scriptContextRegistry, scriptSettings, new ScriptMetrics()); - templateService = new InternalTemplateService(scriptService); + ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(emptyList()); + MustacheScriptEngineService mustache = new MustacheScriptEngineService(); + ScriptSettings scriptSettings = new ScriptSettings(new ScriptEngineRegistry(emptyList()), + mustache, scriptContextRegistry); + + org.elasticsearch.script.TemplateService esTemplateService = + new org.elasticsearch.script.TemplateService(settings, new Environment(settings), + null, mustache, scriptContextRegistry, + scriptSettings, new ScriptMetrics()); + templateService = new InternalTemplateService(esTemplateService); } } diff --git a/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/10_pipeline_with_mustache_templates.yaml b/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/10_pipeline_with_mustache_templates.yaml index 0e54ff0b7ad59..848950a78f685 100644 --- a/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/10_pipeline_with_mustache_templates.yaml +++ b/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/10_pipeline_with_mustache_templates.yaml @@ -349,4 +349,4 @@ } - match: { error.header.processor_type: "set" } - match: { error.type: "general_script_exception" } - - match: { error.reason: "Failed to compile inline script [{{#join}}{{/join}}] using lang [mustache]" } + - match: { error.reason: "Failed to compile [{{#join}}{{/join}}]" } From 6b36b43f6f0348a066b85cda36e5aecaf5f75249 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 24 Mar 2017 11:31:18 -0400 Subject: [PATCH 09/37] Docs --- .../org/elasticsearch/index/IndexService.java | 2 +- .../elasticsearch/script/CachingCompiler.java | 31 +++++++++++++++++-- .../script/StoredScriptSource.java | 2 +- 3 files changed, 30 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 7420ced295634..6baf4e28df9e5 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -503,7 +503,7 @@ public ScriptService getScriptService() { * The {@link TemplateService} to use for this index. */ public TemplateService getTemplateService() { - return null; + return templateService; } List getIndexOperationListeners() { // pkg private for testing diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 3cf190e4c75a1..2f776352e4bca 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -41,7 +41,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.env.Environment; -import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.watcher.FileChangesListener; import org.elasticsearch.watcher.FileWatcher; import org.elasticsearch.watcher.ResourceWatcherService; @@ -122,8 +121,16 @@ public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environ * type is not supported. */ protected abstract CacheKeyT cacheKeyForFile(String baseName, String extension); + + /** + * Build the cache key for a script (or template) stored in the cluster state. + */ protected abstract CacheKeyT cacheKeyFromClusterState(StoredScriptSource scriptMetadata); + /** + * Lookup a stored script (or template) from the cluster state, returning null if it is not + * found. + */ protected abstract StoredScriptSource lookupStoredScript(ClusterState clusterState, CacheKeyT cacheKey); @@ -131,7 +138,7 @@ protected abstract StoredScriptSource lookupStoredScript(ClusterState clusterSta * Are any script contexts enabled for the given {@code cacheKey} and {@code scriptType}? Used * to reject compilation if all script contexts are disabled and produce a nice error message * earlier rather than later. - */ // NOCOMMIT make sure we have tests for cases where we use this (files, cluster state, ?) + */ protected abstract boolean areAnyScriptContextsEnabled(CacheKeyT cacheKey, ScriptType scriptType); @@ -146,8 +153,26 @@ protected abstract void checkCanExecuteScript(CacheKeyT cacheKey, ScriptType scr */ protected abstract void checkCompilationLimit(); - // NOCOMMIT document + /** + * Compile an inline or stored script (or template), throwing an + * {@link IllegalArgumentException} if there is a problem. + * + * @param scriptType whether the script is inline or stored + * @param cacheKey the identifier in the cache. Note that this must contain the script's + * (or template's) source. + * @return the script (or template) compiled + */ protected abstract CompiledScript compile(ScriptType scriptType, CacheKeyT cacheKey); + + /** + * Compile a file script (or template), throwing an {@link IllegalArgumentException} if there is + * a problem. + * + * @param cacheKey identifier in the cache as built by {@link #cacheKeyForFile(String, String)} + * @param body body of the file + * @param file path to the file for passing to the compiler in case it wants it + * @return the script (or template) compiled + */ protected abstract CompiledScript compileFileScript(CacheKeyT cacheKey, String body, Path file); public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, diff --git a/core/src/main/java/org/elasticsearch/script/StoredScriptSource.java b/core/src/main/java/org/elasticsearch/script/StoredScriptSource.java index 11b7821390847..c6b13baa3e311 100644 --- a/core/src/main/java/org/elasticsearch/script/StoredScriptSource.java +++ b/core/src/main/java/org/elasticsearch/script/StoredScriptSource.java @@ -49,7 +49,7 @@ import java.util.Objects; /** - * {@link StoredScriptSource} represents user-defined parameters for a script + * {@link StoredScriptSource} represents user-defined parameters for a script (or template) * saved in the {@link ClusterState}. */ public class StoredScriptSource extends AbstractDiffable implements Writeable, ToXContent { From 1471ebd9ab2ab8f4f917f8108aed7efa2745203d Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 24 Mar 2017 12:19:27 -0400 Subject: [PATCH 10/37] No more NOCOMMITs! --- .../resources/checkstyle_suppressions.xml | 1 - .../java/org/elasticsearch/node/Node.java | 2 +- .../elasticsearch/script/CachingCompiler.java | 8 +- .../elasticsearch/script/ScriptPermits.java | 2 +- .../elasticsearch/script/ScriptService.java | 8 +- .../elasticsearch/script/TemplateService.java | 15 +- .../script/ScriptContextTests.java | 131 --------------- .../script/ScriptPermitsTests.java | 150 +++++++++++++++++- .../script/ScriptSettingsTests.java | 25 +-- 9 files changed, 177 insertions(+), 165 deletions(-) delete mode 100644 core/src/test/java/org/elasticsearch/script/ScriptContextTests.java diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index a52b2c9039670..a96b6aad53f7f 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -3047,7 +3047,6 @@ - diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 90741ae06ebfb..042dd6f6e5256 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -332,7 +332,7 @@ protected Node(final Environment environment, Collection getCustomNameResolvers(pluginsService.filterPlugins(DiscoveryPlugin.class))); final ClusterService clusterService = new ClusterService(settings, settingsModule.getClusterSettings(), threadPool, localNodeFactory::getNode); - clusterService.addListener(scriptModule.getScriptService()); // NOCOMMIT why not do this in the service's ctor? + clusterService.addListener(scriptModule.getScriptService()); clusterService.addListener(scriptModule.getTemplateService()); resourcesToClose.add(clusterService); final IngestService ingestService = new IngestService(settings, threadPool, diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 2f776352e4bca..f1cc579c8c5a1 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -79,7 +79,7 @@ public abstract class CachingCompiler implements ClusterStateListener private volatile ClusterState clusterState; - public CachingCompiler(Settings settings, ScriptSettings scriptSettings, Environment env, + public CachingCompiler(Settings settings, Environment env, ResourceWatcherService resourceWatcherService, ScriptMetrics scriptMetrics, String type) throws IOException { int cacheMaxSize = ScriptService.SCRIPT_CACHE_SIZE_SETTING.get(settings); @@ -143,9 +143,9 @@ protected abstract boolean areAnyScriptContextsEnabled(CacheKeyT cacheKey, ScriptType scriptType); /** - * Check if a script can be executed. + * Check if a script (or template) can be executed in a particular context. */ - protected abstract void checkCanExecuteScript(CacheKeyT cacheKey, ScriptType scriptType, + protected abstract void checkContextPermissions(CacheKeyT cacheKey, ScriptType scriptType, ScriptContext scriptContext); /** @@ -185,7 +185,7 @@ public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, } // Validate that we can execute the script - checkCanExecuteScript(cacheKey, scriptType, scriptContext); + checkContextPermissions(cacheKey, scriptType, scriptContext); // Lookup file scripts from the map we maintain by watching the directory if (scriptType == ScriptType.FILE) { diff --git a/core/src/main/java/org/elasticsearch/script/ScriptPermits.java b/core/src/main/java/org/elasticsearch/script/ScriptPermits.java index bfe86cdf834b1..a5efa2631f334 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptPermits.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptPermits.java @@ -100,7 +100,7 @@ public void checkCompilationLimit() { } } - public boolean canExecuteScript(String lang, ScriptType scriptType, + public boolean checkContextPermissions(String lang, ScriptType scriptType, ScriptContext scriptContext) { assert lang != null; if (scriptContextRegistry.isSupportedContext(scriptContext) == false) { diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index c87f068caea37..53e54ffd97a7d 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -105,7 +105,7 @@ public ScriptService(Settings settings, Environment env, ResourceWatcherService maxScriptSizeInBytes = ScriptService.SCRIPT_MAX_SIZE_IN_BYTES.get(settings); this.scriptMetrics = scriptMetrics; this.scriptPermits = new ScriptPermits(settings, scriptSettings, scriptContextRegistry); - this.compiler = new CachingCompiler(settings, scriptSettings, env, resourceWatcherService, scriptMetrics, "script") { + this.compiler = new CachingCompiler(settings, env, resourceWatcherService, scriptMetrics, "script") { @Override protected CacheKey cacheKeyForFile(String baseName, String extension) { if (extension.equals("mustache")) { @@ -169,7 +169,7 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, Cache @Override protected boolean areAnyScriptContextsEnabled(CacheKey cacheKey, ScriptType scriptType) { for (ScriptContext scriptContext : scriptContextRegistry.scriptContexts()) { - if (scriptPermits.canExecuteScript(cacheKey.lang, scriptType, scriptContext)) { + if (scriptPermits.checkContextPermissions(cacheKey.lang, scriptType, scriptContext)) { return true; } } @@ -177,7 +177,7 @@ protected boolean areAnyScriptContextsEnabled(CacheKey cacheKey, ScriptType scri } @Override - protected void checkCanExecuteScript(CacheKey cacheKey, ScriptType scriptType, ScriptContext scriptContext) { + protected void checkContextPermissions(CacheKey cacheKey, ScriptType scriptType, ScriptContext scriptContext) { if (isLangSupported(cacheKey.lang) == false) { throw new IllegalArgumentException("script_lang not supported [" + cacheKey.lang + "]"); } @@ -189,7 +189,7 @@ protected void checkCanExecuteScript(CacheKey cacheKey, ScriptType scriptType, S throw new UnsupportedOperationException("scripts of type [" + scriptType + "]," + " operation [" + scriptContext.getKey() + "] and lang [" + cacheKey.lang + "] are not supported"); } - if (scriptPermits.canExecuteScript(cacheKey.lang, scriptType, scriptContext) == false) { + if (scriptPermits.checkContextPermissions(cacheKey.lang, scriptType, scriptContext) == false) { throw new IllegalStateException("scripts of type [" + scriptType + "]," + " operation [" + scriptContext.getKey() + "] and lang [" + cacheKey.lang + "] are disabled"); } diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index d072872327a36..614671db191f7 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -56,8 +56,8 @@ public TemplateService(Settings settings, Environment env, this.backend = backend; this.scriptPermits = new ScriptPermits(settings, scriptSettings, scriptContextRegistry); - this.compiler = new CachingCompiler(settings, scriptSettings, env, - resourceWatcherService, scriptMetrics, "template") { + this.compiler = new CachingCompiler(settings, env, resourceWatcherService, + scriptMetrics, "template") { @Override protected String cacheKeyForFile(String baseName, String extension) { if (false == backend.getType().equals(extension)) { @@ -111,7 +111,7 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, @Override protected boolean areAnyScriptContextsEnabled(String cacheKey, ScriptType scriptType) { for (ScriptContext scriptContext : scriptContextRegistry.scriptContexts()) { - if (scriptPermits.canExecuteScript(backend.getType(), scriptType, + if (scriptPermits.checkContextPermissions(backend.getType(), scriptType, scriptContext)) { return true; } @@ -120,13 +120,12 @@ protected boolean areAnyScriptContextsEnabled(String cacheKey, ScriptType script } @Override - protected void checkCanExecuteScript(String cacheKey, ScriptType scriptType, + protected void checkContextPermissions(String cacheKey, ScriptType scriptType, ScriptContext scriptContext) { - if (scriptPermits.canExecuteScript(backend.getType(), scriptType, + if (scriptPermits.checkContextPermissions(backend.getType(), scriptType, scriptContext) == false) { - throw new IllegalStateException("scripts of type [" + scriptType + "]," - + " operation [" + scriptContext.getKey() + "] and lang [" - + backend.getType() + "] are disabled"); + throw new IllegalStateException("templates of [" + scriptType + "]," + + " operation [" + scriptContext.getKey() + "] are disabled"); } } diff --git a/core/src/test/java/org/elasticsearch/script/ScriptContextTests.java b/core/src/test/java/org/elasticsearch/script/ScriptContextTests.java deleted file mode 100644 index 94450d748562e..0000000000000 --- a/core/src/test/java/org/elasticsearch/script/ScriptContextTests.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * 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.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.env.Environment; -import org.elasticsearch.test.ESTestCase; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import static org.hamcrest.Matchers.containsString; - -public class ScriptContextTests extends ESTestCase { - - private static final String PLUGIN_NAME = "testplugin"; - - ScriptService makeScriptService() throws Exception { - Settings settings = Settings.builder() - .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) - // no file watching, so we don't need a ResourceWatcherService - .put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), "false") - .put("script." + PLUGIN_NAME + "_custom_globally_disabled_op", "false") - .put("script.engine." + MockScriptEngine.NAME + ".inline." + PLUGIN_NAME + "_custom_exp_disabled_op", "false") - .build(); - - MockScriptEngine scriptEngine = new MockScriptEngine(MockScriptEngine.NAME, Collections.singletonMap("1", script -> "1")); - ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.singletonList(scriptEngine)); - List customContexts = Arrays.asList( - new ScriptContext.Plugin(PLUGIN_NAME, "custom_op"), - new ScriptContext.Plugin(PLUGIN_NAME, "custom_exp_disabled_op"), - new ScriptContext.Plugin(PLUGIN_NAME, "custom_globally_disabled_op")); - ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(customContexts); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); // NOCOMMIT remove null and test - ScriptService scriptService = new ScriptService(settings, new Environment(settings), null, scriptEngineRegistry, - scriptContextRegistry, scriptSettings, new ScriptMetrics()); - - ClusterState empty = ClusterState.builder(new ClusterName("_name")).build(); - ScriptMetaData smd = empty.metaData().custom(ScriptMetaData.TYPE); - smd = ScriptMetaData.putStoredScript(smd, "1", new StoredScriptSource(MockScriptEngine.NAME, "1", Collections.emptyMap())); - MetaData.Builder mdb = MetaData.builder(empty.getMetaData()).putCustom(ScriptMetaData.TYPE, smd); - ClusterState stored = ClusterState.builder(empty).metaData(mdb).build(); - scriptService.clusterChanged(new ClusterChangedEvent("test", stored, empty)); - - return scriptService; - } - - public void testCustomGlobalScriptContextSettings() throws Exception { - ScriptService scriptService = makeScriptService(); - for (ScriptType scriptType : ScriptType.values()) { - try { - Script script = new Script(scriptType, MockScriptEngine.NAME, "1", Collections.emptyMap()); - scriptService.compile(script, new ScriptContext.Plugin(PLUGIN_NAME, "custom_globally_disabled_op")); - fail("script compilation should have been rejected"); - } catch (IllegalStateException e) { - assertThat(e.getMessage(), containsString("scripts of type [" + scriptType + "], operation [" + PLUGIN_NAME + "_custom_globally_disabled_op] and lang [" + MockScriptEngine.NAME + "] are disabled")); - } - } - } - - public void testCustomScriptContextSettings() throws Exception { - ScriptService scriptService = makeScriptService(); - Script script = new Script(ScriptType.INLINE, MockScriptEngine.NAME, "1", Collections.emptyMap()); - try { - scriptService.compile(script, new ScriptContext.Plugin(PLUGIN_NAME, "custom_exp_disabled_op")); - fail("script compilation should have been rejected"); - } catch (IllegalStateException e) { - assertTrue(e.getMessage(), e.getMessage().contains("scripts of type [inline], operation [" + PLUGIN_NAME + "_custom_exp_disabled_op] and lang [" + MockScriptEngine.NAME + "] are disabled")); - } - - // still works for other script contexts - assertNotNull(scriptService.compile(script, ScriptContext.Standard.AGGS)); - assertNotNull(scriptService.compile(script, ScriptContext.Standard.SEARCH)); - assertNotNull(scriptService.compile(script, new ScriptContext.Plugin(PLUGIN_NAME, "custom_op"))); - } - - public void testUnknownPluginScriptContext() throws Exception { - ScriptService scriptService = makeScriptService(); - for (ScriptType scriptType : ScriptType.values()) { - try { - Script script = new Script(scriptType, MockScriptEngine.NAME, "1", Collections.emptyMap()); - scriptService.compile(script, new ScriptContext.Plugin(PLUGIN_NAME, "unknown")); - fail("script compilation should have been rejected"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage(), e.getMessage().contains("script context [" + PLUGIN_NAME + "_unknown] not supported")); - } - } - } - - public void testUnknownCustomScriptContext() throws Exception { - ScriptContext context = new ScriptContext() { - @Override - public String getKey() { - return "test"; - } - }; - ScriptService scriptService = makeScriptService(); - for (ScriptType scriptType : ScriptType.values()) { - try { - Script script = new Script(scriptType, MockScriptEngine.NAME, "1", Collections.emptyMap()); - scriptService.compile(script, context); - fail("script compilation should have been rejected"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage(), e.getMessage().contains("script context [test] not supported")); - } - } - } - -} diff --git a/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java b/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java index 01ea513af02df..04532d0fe748d 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java @@ -20,17 +20,55 @@ package org.elasticsearch.script; import org.elasticsearch.common.breaker.CircuitBreakingException; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.test.ESTestCase; +import org.junit.Before; -import static java.util.Collections.emptyList; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; public class ScriptPermitsTests extends ESTestCase { + private static final String PLUGIN_NAME = "testplugin"; + + private ScriptPermits permits; + + @Before + public void makePermist() throws Exception { + Settings settings = Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) + // no file watching, so we don't need a ResourceWatcherService + .put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), "false") + .put("script." + PLUGIN_NAME + "_custom_globally_disabled_op", "false") + .put("script.engine." + MockScriptEngine.NAME + ".inline." + + PLUGIN_NAME + "_custom_exp_disabled_op", "false") + .put("script.engine.mock_template.inline." + + PLUGIN_NAME + "_custom_exp_disabled_op", "false") + .build(); + + TemplateService.Backend templateBackend = new MockTemplateBackend(); + MockScriptEngine scriptEngine = new MockScriptEngine(MockScriptEngine.NAME, + singletonMap("1", script -> "1")); + ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry( + singletonList(scriptEngine)); + List customContexts = Arrays.asList( + new ScriptContext.Plugin(PLUGIN_NAME, "custom_op"), + new ScriptContext.Plugin(PLUGIN_NAME, "custom_exp_disabled_op"), + new ScriptContext.Plugin(PLUGIN_NAME, "custom_globally_disabled_op")); + ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(customContexts); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, + templateBackend, scriptContextRegistry); + permits = new ScriptPermits(settings, scriptSettings, scriptContextRegistry); + } + public void testCompilationCircuitBreaking() throws Exception { - ScriptContextRegistry contextRegistry = new ScriptContextRegistry(emptyList()); - ScriptSettings scriptSettings = new ScriptSettings(new ScriptEngineRegistry(emptyList()), - null, contextRegistry); - ScriptPermits permits = new ScriptPermits(Settings.EMPTY, scriptSettings, contextRegistry); permits.setMaxCompilationsPerMinute(1); permits.checkCompilationLimit(); // should pass expectThrows(CircuitBreakingException.class, () -> permits.checkCompilationLimit()); @@ -52,4 +90,106 @@ public void testCompilationCircuitBreaking() throws Exception { permits.checkCompilationLimit(); } } + + public void testCustomGlobalScriptContextSettings() throws Exception { + for (ScriptType scriptType : ScriptType.values()) { + assertFalse(permits.checkContextPermissions(MockScriptEngine.NAME, scriptType, + new ScriptContext.Plugin(PLUGIN_NAME, "custom_globally_disabled_op"))); + assertFalse(permits.checkContextPermissions("mock_template", scriptType, + new ScriptContext.Plugin(PLUGIN_NAME, "custom_globally_disabled_op"))); + } + } + + public void testCustomScriptContextSettings() throws Exception { + assertFalse(permits.checkContextPermissions(MockScriptEngine.NAME, ScriptType.INLINE, + new ScriptContext.Plugin(PLUGIN_NAME, "custom_exp_disabled_op"))); + assertFalse(permits.checkContextPermissions("mock_template", ScriptType.INLINE, + new ScriptContext.Plugin(PLUGIN_NAME, "custom_exp_disabled_op"))); + + // still works for other script contexts + assertTrue(permits.checkContextPermissions(MockScriptEngine.NAME, ScriptType.INLINE, + ScriptContext.Standard.AGGS)); + assertTrue(permits.checkContextPermissions(MockScriptEngine.NAME, ScriptType.INLINE, + ScriptContext.Standard.SEARCH)); + assertTrue(permits.checkContextPermissions(MockScriptEngine.NAME, ScriptType.INLINE, + new ScriptContext.Plugin(PLUGIN_NAME, "custom_op"))); + assertTrue(permits.checkContextPermissions("mock_template", ScriptType.INLINE, + ScriptContext.Standard.AGGS)); + assertTrue(permits.checkContextPermissions("mock_template", ScriptType.INLINE, + ScriptContext.Standard.SEARCH)); + assertTrue(permits.checkContextPermissions("mock_template", ScriptType.INLINE, + new ScriptContext.Plugin(PLUGIN_NAME, "custom_op"))); + } + + public void testUnknownPluginScriptContext() throws Exception { + for (ScriptType scriptType : ScriptType.values()) { + Exception e = expectThrows(IllegalArgumentException.class, () -> + permits.checkContextPermissions(MockScriptEngine.NAME, scriptType, + new ScriptContext.Plugin(PLUGIN_NAME, "unknown"))); + assertEquals("script context [testplugin_unknown] not supported", e.getMessage()); + e = expectThrows(IllegalArgumentException.class, () -> + permits.checkContextPermissions("mock_template", scriptType, + new ScriptContext.Plugin(PLUGIN_NAME, "unknown"))); + assertEquals("script context [testplugin_unknown] not supported", e.getMessage()); + } + } + + public void testUnknownCustomScriptContext() throws Exception { + ScriptContext context = new ScriptContext() { + @Override + public String getKey() { + return "test"; + } + }; + for (ScriptType scriptType : ScriptType.values()) { + Exception e = expectThrows(IllegalArgumentException.class, () -> + permits.checkContextPermissions(MockScriptEngine.NAME, scriptType, context)); + assertEquals("script context [test] not supported", e.getMessage()); + e = expectThrows(IllegalArgumentException.class, () -> + permits.checkContextPermissions("mock_template", scriptType, context)); + assertEquals("script context [test] not supported", e.getMessage()); + } + } + + private static class MockTemplateBackend implements TemplateService.Backend { + @Override + public String getType() { + return "mock_template"; + } + + @Override + public String getExtension() { + return "mock_template"; + } + + @Override + public Object compile(String scriptName, String scriptSource, Map params) { + return scriptSource; + } + + @Override + public ExecutableScript executable(CompiledScript compiledScript, + Map vars) { + return new ExecutableScript() { + @Override + public void setNextVar(String name, Object value) { + } + + @Override + public Object run() { + return new BytesArray((String) compiledScript.compiled()); + } + }; + } + + @Override + public SearchScript search(CompiledScript compiledScript, SearchLookup lookup, + Map vars) { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException { + } + } } diff --git a/core/src/test/java/org/elasticsearch/script/ScriptSettingsTests.java b/core/src/test/java/org/elasticsearch/script/ScriptSettingsTests.java index 510b35e920fb6..a8e4699945271 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptSettingsTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptSettingsTests.java @@ -25,29 +25,32 @@ import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.test.ESTestCase; -import java.util.Collections; import java.util.Iterator; import java.util.Map; +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonList; import static org.hamcrest.Matchers.equalTo; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; public class ScriptSettingsTests extends ESTestCase { - public void testSettingsAreProperlyPropogated() { - ScriptEngineRegistry scriptEngineRegistry = - new ScriptEngineRegistry(Collections.singletonList(new CustomScriptEngineService())); - ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList()); + ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry( + singletonList(new CustomScriptEngineService())); + ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(emptyList()); TemplateService.Backend templateBackend = mock(TemplateService.Backend.class); when(templateBackend.getType()).thenReturn("test_template_backend"); - ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, templateBackend, scriptContextRegistry); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, templateBackend, + scriptContextRegistry); boolean enabled = randomBoolean(); Settings s = Settings.builder().put("script.inline", enabled).build(); - for (Iterator> iter = scriptSettings.getScriptLanguageSettings().iterator(); iter.hasNext();) { + for (Iterator> iter = scriptSettings.getScriptLanguageSettings() + .iterator(); iter.hasNext();) { Setting setting = iter.next(); if (setting.getKey().endsWith(".inline")) { - assertThat("inline settings should have propagated", setting.get(s), equalTo(enabled)); + assertThat("inline settings should have propagated", setting.get(s), + equalTo(enabled)); assertThat(setting.getDefaultRaw(s), equalTo(Boolean.toString(enabled))); } } @@ -73,12 +76,14 @@ public Object compile(String scriptName, String scriptSource, Map vars) { + public ExecutableScript executable(CompiledScript compiledScript, + @Nullable Map vars) { return null; } @Override - public SearchScript search(CompiledScript compiledScript, SearchLookup lookup, @Nullable Map vars) { + public SearchScript search(CompiledScript compiledScript, SearchLookup lookup, + @Nullable Map vars) { return null; } From b8f0f08f7cc99d489dee6bb08ec986e794dd9029 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 24 Mar 2017 12:55:09 -0400 Subject: [PATCH 11/37] Last nocommit for real this time --- .../script/ScriptModesTests.java | 67 ++++++++++++------- .../script/ScriptPermitsTests.java | 2 +- 2 files changed, 43 insertions(+), 26 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java b/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java index ce0f1b8b66d41..3d5c82425e2ae 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptModesTests.java @@ -21,24 +21,24 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.script.ScriptPermitsTests.MockTemplateBackend; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.test.ESTestCase; import org.junit.After; import org.junit.Before; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; +import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; import static org.elasticsearch.common.util.set.Sets.newHashSet; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.Matchers.containsString; -// TODO: this needs to be a base test class, and all scripting engines extend it public class ScriptModesTests extends ESTestCase { ScriptSettings scriptSettings; ScriptContextRegistry scriptContextRegistry; @@ -65,10 +65,11 @@ public void setupScriptEngines() { scriptContexts = scriptContextRegistry.scriptContexts().toArray(new ScriptContext[scriptContextRegistry.scriptContexts().size()]); scriptEngines = buildScriptEnginesByLangMap(newHashSet( //add the native engine just to make sure it gets filtered out - new NativeScriptEngineService(Settings.EMPTY, Collections.emptyMap()), + new NativeScriptEngineService(Settings.EMPTY, emptyMap()), new CustomScriptEngineService())); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(scriptEngines.values()); - scriptSettings = new ScriptSettings(scriptEngineRegistry, null, scriptContextRegistry); // NOCOMMIT remove null and test + scriptSettings = new ScriptSettings(scriptEngineRegistry, new MockTemplateBackend(), + scriptContextRegistry); checkedSettings = new HashSet<>(); assertAllSettingsWereChecked = true; assertScriptModesNonNull = true; @@ -87,6 +88,7 @@ public void assertAllSettingsWereChecked() { assertThat(scriptModes, notNullValue()); int numberOfSettings = ScriptType.values().length * scriptContextRegistry.scriptContexts().size(); numberOfSettings += 3; // for top-level inline/store/file settings + numberOfSettings *= 2; // Once for the script engine, once for the template engine assertThat(scriptModes.scriptEnabled.size(), equalTo(numberOfSettings)); if (assertAllSettingsWereChecked) { assertThat(checkedSettings.size(), equalTo(numberOfSettings)); @@ -96,8 +98,10 @@ public void assertAllSettingsWereChecked() { public void testDefaultSettings() { this.scriptModes = new ScriptModes(scriptSettings, Settings.EMPTY); - assertScriptModesAllOps(true, ScriptType.FILE); - assertScriptModesAllOps(false, ScriptType.STORED, ScriptType.INLINE); + assertScriptModesAllOps("custom", true, ScriptType.FILE); + assertScriptModesAllOps("custom", false, ScriptType.STORED, ScriptType.INLINE); + assertScriptModesAllOps("mock_template", true, ScriptType.FILE, ScriptType.STORED, + ScriptType.INLINE); } public void testMissingSetting() { @@ -130,16 +134,20 @@ public void testScriptTypeGenericSettings() { this.scriptModes = new ScriptModes(scriptSettings, builder.build()); for (int i = 0; i < randomInt; i++) { - assertScriptModesAllOps(randomScriptModes[i], randomScriptTypes[i]); + assertScriptModesAllOps("custom", randomScriptModes[i], randomScriptTypes[i]); + assertScriptModesAllOps("mock_template", randomScriptModes[i], randomScriptTypes[i]); } if (randomScriptTypesSet.contains(ScriptType.FILE) == false) { - assertScriptModesAllOps(true, ScriptType.FILE); + assertScriptModesAllOps("custom", true, ScriptType.FILE); + assertScriptModesAllOps("mock_template", true, ScriptType.FILE); } if (randomScriptTypesSet.contains(ScriptType.STORED) == false) { - assertScriptModesAllOps(false, ScriptType.STORED); + assertScriptModesAllOps("custom", false, ScriptType.STORED); + assertScriptModesAllOps("mock_template", true, ScriptType.STORED); } if (randomScriptTypesSet.contains(ScriptType.INLINE) == false) { - assertScriptModesAllOps(false, ScriptType.INLINE); + assertScriptModesAllOps("custom", false, ScriptType.INLINE); + assertScriptModesAllOps("mock_template", true, ScriptType.INLINE); } } @@ -162,12 +170,14 @@ public void testScriptContextGenericSettings() { this.scriptModes = new ScriptModes(scriptSettings, builder.build()); for (int i = 0; i < randomInt; i++) { - assertScriptModesAllTypes(randomScriptModes[i], randomScriptContexts[i]); + assertScriptModesAllTypes("custom", randomScriptModes[i], randomScriptContexts[i]); + assertScriptModesAllTypes("mock_template", randomScriptModes[i], randomScriptContexts[i]); } ScriptContext[] complementOf = complementOf(randomScriptContexts); - assertScriptModes(true, new ScriptType[]{ScriptType.FILE}, complementOf); - assertScriptModes(false, new ScriptType[]{ScriptType.STORED, ScriptType.INLINE}, complementOf); + assertScriptModes("custom", true, new ScriptType[] {ScriptType.FILE}, complementOf); + assertScriptModes("custom", false, new ScriptType[] {ScriptType.STORED, ScriptType.INLINE}, complementOf); + assertScriptModes("mock_template", true, ScriptType.values(), complementOf); } public void testConflictingScriptTypeAndOpGenericSettings() { @@ -178,29 +188,36 @@ public void testConflictingScriptTypeAndOpGenericSettings() { .put("script.inline", "true"); //operations generic settings have precedence over script type generic settings this.scriptModes = new ScriptModes(scriptSettings, builder.build()); - assertScriptModesAllTypes(false, scriptContext); + assertScriptModesAllTypes("custom", false, scriptContext); + assertScriptModesAllTypes("mock_template", false, scriptContext); ScriptContext[] complementOf = complementOf(scriptContext); - assertScriptModes(true, new ScriptType[]{ScriptType.FILE, ScriptType.STORED}, complementOf); - assertScriptModes(true, new ScriptType[]{ScriptType.INLINE}, complementOf); + assertScriptModes("custom", true, new ScriptType[] {ScriptType.FILE, ScriptType.STORED}, complementOf); + assertScriptModes("custom", true, new ScriptType[] {ScriptType.INLINE}, complementOf); + assertScriptModes("mock_template", true, ScriptType.values(), complementOf); } - private void assertScriptModesAllOps(boolean expectedScriptEnabled, ScriptType... scriptTypes) { - assertScriptModes(expectedScriptEnabled, scriptTypes, scriptContexts); + private void assertScriptModesAllOps(String lang, boolean expectedScriptEnabled, + ScriptType... scriptTypes) { + assertScriptModes(lang, expectedScriptEnabled, scriptTypes, scriptContexts); } - private void assertScriptModesAllTypes(boolean expectedScriptEnabled, ScriptContext... scriptContexts) { - assertScriptModes(expectedScriptEnabled, ScriptType.values(), scriptContexts); + private void assertScriptModesAllTypes(String lang, boolean expectedScriptEnabled, + ScriptContext... scriptContexts) { + assertScriptModes(lang, expectedScriptEnabled, ScriptType.values(), scriptContexts); } - private void assertScriptModes(boolean expectedScriptEnabled, ScriptType[] scriptTypes, ScriptContext... scriptContexts) { + private void assertScriptModes(String lang, boolean expectedScriptEnabled, + ScriptType[] scriptTypes, ScriptContext... scriptContexts) { assert scriptTypes.length > 0; assert scriptContexts.length > 0; for (ScriptType scriptType : scriptTypes) { - checkedSettings.add("script.engine.custom." + scriptType); + checkedSettings.add("script.engine." + lang + "." + scriptType); for (ScriptContext scriptContext : scriptContexts) { - assertThat("custom." + scriptType + "." + scriptContext.getKey() + " doesn't have the expected value", - scriptModes.getScriptEnabled("custom", scriptType, scriptContext), equalTo(expectedScriptEnabled)); - checkedSettings.add("custom." + scriptType + "." + scriptContext); + assertEquals(lang + "." + scriptType + "." + scriptContext.getKey() + + " doesn't have the expected value", + expectedScriptEnabled, + scriptModes.getScriptEnabled(lang, scriptType, scriptContext)); + checkedSettings.add(lang + "." + scriptType + "." + scriptContext); } } } diff --git a/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java b/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java index 04532d0fe748d..74b375fd2d319 100644 --- a/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java +++ b/core/src/test/java/org/elasticsearch/script/ScriptPermitsTests.java @@ -151,7 +151,7 @@ public String getKey() { } } - private static class MockTemplateBackend implements TemplateService.Backend { + static class MockTemplateBackend implements TemplateService.Backend { @Override public String getType() { return "mock_template"; From 31d733f4647877ca75a634ea2e763f1808e6ddaf Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 24 Mar 2017 16:55:49 -0400 Subject: [PATCH 12/37] TemplateService in createComponents --- .../main/java/org/elasticsearch/node/Node.java | 10 ++++++---- .../java/org/elasticsearch/plugins/Plugin.java | 10 +++++++--- .../org/elasticsearch/script/ScriptModule.java | 16 +++++++++------- .../discovery/file/FileBasedDiscoveryPlugin.java | 2 ++ 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 042dd6f6e5256..dd9dc9de0eafc 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -395,10 +395,12 @@ protected Node(final Environment environment, Collection threadPool, settingsModule.getIndexScopedSettings(), circuitBreakerService, bigArrays, scriptModule.getScriptService(), scriptModule.getTemplateService(), client, metaStateService); - Collection pluginComponents = pluginsService.filterPlugins(Plugin.class).stream() - .flatMap(p -> p.createComponents(client, clusterService, threadPool, resourceWatcherService, - scriptModule.getScriptService(), xContentRegistry).stream()) - .collect(Collectors.toList()); + Collection pluginComponents = pluginsService.filterPlugins(Plugin.class) + .stream() + .flatMap(p -> p.createComponents(client, clusterService, threadPool, + resourceWatcherService, scriptModule.getScriptService(), + scriptModule.getTemplateService(), xContentRegistry).stream()) + .collect(Collectors.toList()); Collection>> customMetaDataUpgraders = pluginsService.filterPlugins(Plugin.class).stream() .map(Plugin::getCustomMetaDataUpgrader) diff --git a/core/src/main/java/org/elasticsearch/plugins/Plugin.java b/core/src/main/java/org/elasticsearch/plugins/Plugin.java index 87c5ef9a8c694..fe207017e4b5a 100644 --- a/core/src/main/java/org/elasticsearch/plugins/Plugin.java +++ b/core/src/main/java/org/elasticsearch/plugins/Plugin.java @@ -41,6 +41,7 @@ import org.elasticsearch.repositories.RepositoriesModule; import org.elasticsearch.script.ScriptModule; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.TemplateService; import org.elasticsearch.search.SearchModule; import org.elasticsearch.threadpool.ExecutorBuilder; import org.elasticsearch.threadpool.ThreadPool; @@ -102,10 +103,13 @@ public Collection> getGuiceServiceClasses() * @param threadPool A service to allow retrieving an executor to run an async action * @param resourceWatcherService A service to watch for changes to node local files * @param scriptService A service to allow running scripts on the local node + * @param templateService A service to render templates on the local node + * @param xContentRegistry Registry of named XContent parsers */ - public Collection createComponents(Client client, ClusterService clusterService, ThreadPool threadPool, - ResourceWatcherService resourceWatcherService, ScriptService scriptService, - NamedXContentRegistry xContentRegistry) { + public Collection createComponents(Client client, ClusterService clusterService, + ThreadPool threadPool, ResourceWatcherService resourceWatcherService, + ScriptService scriptService, TemplateService templateService, + NamedXContentRegistry xContentRegistry) { return Collections.emptyList(); } diff --git a/core/src/main/java/org/elasticsearch/script/ScriptModule.java b/core/src/main/java/org/elasticsearch/script/ScriptModule.java index 54c1699fe3f39..70faab47afff9 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptModule.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptModule.java @@ -136,27 +136,29 @@ public void registerClusterSettingsListeners(ClusterSettings clusterSettings) { private static class TemplatesUnsupportedBackend implements TemplateService.Backend { @Override public String getType() { - throw new UnsupportedOperationException("templates are unsupported"); + throw new UnsupportedOperationException("no template backend installed"); } @Override public String getExtension() { - throw new UnsupportedOperationException("templates are unsupported"); + throw new UnsupportedOperationException("no template backend installed"); } @Override public Object compile(String scriptName, String scriptSource, Map params) { - throw new UnsupportedOperationException("templates are unsupported"); + throw new UnsupportedOperationException("no template backend installed"); } @Override - public ExecutableScript executable(CompiledScript compiledScript, Map vars) { - throw new UnsupportedOperationException("templates are unsupported"); + public ExecutableScript executable(CompiledScript compiledScript, + Map vars) { + throw new UnsupportedOperationException("no template backend installed"); } @Override - public SearchScript search(CompiledScript compiledScript, SearchLookup lookup, Map vars) { - throw new UnsupportedOperationException("templates are unsupported"); + public SearchScript search(CompiledScript compiledScript, SearchLookup lookup, + Map vars) { + throw new UnsupportedOperationException("no template backend installed"); } @Override diff --git a/plugins/discovery-file/src/main/java/org/elasticsearch/discovery/file/FileBasedDiscoveryPlugin.java b/plugins/discovery-file/src/main/java/org/elasticsearch/discovery/file/FileBasedDiscoveryPlugin.java index d809fd3fa885e..1b7b06b4d3820 100644 --- a/plugins/discovery-file/src/main/java/org/elasticsearch/discovery/file/FileBasedDiscoveryPlugin.java +++ b/plugins/discovery-file/src/main/java/org/elasticsearch/discovery/file/FileBasedDiscoveryPlugin.java @@ -35,6 +35,7 @@ import org.elasticsearch.plugins.DiscoveryPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.TemplateService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.elasticsearch.watcher.ResourceWatcherService; @@ -72,6 +73,7 @@ public Collection createComponents( ThreadPool threadPool, ResourceWatcherService resourceWatcherService, ScriptService scriptService, + TemplateService templateService, NamedXContentRegistry xContentRegistry) { final int concurrentConnects = UnicastZenPing.DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.get(settings); final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(settings, "[file_based_discovery_resolve]"); From 706ca6daf96b26958708f071c7f28fa188fcedb2 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 24 Mar 2017 17:17:56 -0400 Subject: [PATCH 13/37] Many renames --- .../script/mustache/MustachePlugin.java | 28 ++++++++--------- ...n.java => DeleteStoredTemplateAction.java} | 22 +++++++------- ....java => DeleteStoredTemplateRequest.java} | 10 +++---- ...> DeleteStoredTemplateRequestBuilder.java} | 16 +++++----- ...java => DeleteStoredTemplateResponse.java} | 7 +++-- ...tion.java => GetStoredTemplateAction.java} | 22 +++++++------- ...est.java => GetStoredTemplateRequest.java} | 10 +++---- ...a => GetStoredTemplateRequestBuilder.java} | 16 +++++----- ...se.java => GetStoredTemplateResponse.java} | 6 ++-- ...tion.java => PutStoredTemplateAction.java} | 22 +++++++------- ...est.java => PutStoredTemplateRequest.java} | 12 ++++---- ...a => PutStoredTemplateRequestBuilder.java} | 18 +++++------ ...se.java => PutStoredTemplateResponse.java} | 7 ++--- ...va => RestDeleteStoredTemplateAction.java} | 8 ++--- ....java => RestGetStoredTemplateAction.java} | 12 ++++---- ....java => RestPutStoredTemplateAction.java} | 8 ++--- ... TransportDeleteStoredTemplateAction.java} | 26 ++++++++-------- ... => TransportGetStoredTemplateAction.java} | 22 +++++++------- ...ransportPutStoredSearchTemplateAction.java | 22 +++++++------- .../script/mustache/SearchTemplateIT.java | 30 +++++++++---------- 20 files changed, 162 insertions(+), 162 deletions(-) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{DeleteStoredSearchTemplateAction.java => DeleteStoredTemplateAction.java} (62%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{DeleteStoredSearchTemplateRequest.java => DeleteStoredTemplateRequest.java} (89%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{DeleteStoredSearchTemplateRequestBuilder.java => DeleteStoredTemplateRequestBuilder.java} (67%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{PutStoredSearchTemplateResponse.java => DeleteStoredTemplateResponse.java} (88%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{GetStoredSearchTemplateAction.java => GetStoredTemplateAction.java} (63%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{GetStoredSearchTemplateRequest.java => GetStoredTemplateRequest.java} (89%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{GetStoredSearchTemplateRequestBuilder.java => GetStoredTemplateRequestBuilder.java} (67%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{GetStoredSearchTemplateResponse.java => GetStoredTemplateResponse.java} (91%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{PutStoredSearchTemplateAction.java => PutStoredTemplateAction.java} (63%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{PutStoredSearchTemplateRequest.java => PutStoredTemplateRequest.java} (90%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{PutStoredSearchTemplateRequestBuilder.java => PutStoredTemplateRequestBuilder.java} (68%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{DeleteStoredSearchTemplateResponse.java => PutStoredTemplateResponse.java} (87%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{RestDeleteStoredSearchTemplateAction.java => RestDeleteStoredTemplateAction.java} (80%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{RestGetStoredSearchTemplateAction.java => RestGetStoredTemplateAction.java} (85%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{RestPutStoredSearchTemplateAction.java => RestPutStoredTemplateAction.java} (84%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{TransportDeleteStoredSearchTemplateAction.java => TransportDeleteStoredTemplateAction.java} (75%) rename modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/{TransportGetStoredSearchTemplateAction.java => TransportGetStoredTemplateAction.java} (75%) diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java index 95f77de8635a9..90fd885b68148 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustachePlugin.java @@ -34,14 +34,14 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; import org.elasticsearch.script.TemplateService.Backend; -import org.elasticsearch.script.mustache.stored.DeleteStoredSearchTemplateAction; -import org.elasticsearch.script.mustache.stored.GetStoredSearchTemplateAction; -import org.elasticsearch.script.mustache.stored.PutStoredSearchTemplateAction; -import org.elasticsearch.script.mustache.stored.RestDeleteStoredSearchTemplateAction; -import org.elasticsearch.script.mustache.stored.RestGetStoredSearchTemplateAction; -import org.elasticsearch.script.mustache.stored.RestPutStoredSearchTemplateAction; -import org.elasticsearch.script.mustache.stored.TransportDeleteStoredSearchTemplateAction; -import org.elasticsearch.script.mustache.stored.TransportGetStoredSearchTemplateAction; +import org.elasticsearch.script.mustache.stored.DeleteStoredTemplateAction; +import org.elasticsearch.script.mustache.stored.GetStoredTemplateAction; +import org.elasticsearch.script.mustache.stored.PutStoredTemplateAction; +import org.elasticsearch.script.mustache.stored.RestDeleteStoredTemplateAction; +import org.elasticsearch.script.mustache.stored.RestGetStoredTemplateAction; +import org.elasticsearch.script.mustache.stored.RestPutStoredTemplateAction; +import org.elasticsearch.script.mustache.stored.TransportDeleteStoredTemplateAction; +import org.elasticsearch.script.mustache.stored.TransportGetStoredTemplateAction; import org.elasticsearch.script.mustache.stored.TransportPutStoredSearchTemplateAction; import java.util.Arrays; @@ -61,9 +61,9 @@ public Backend getTemplateBackend() { return Arrays.asList( new ActionHandler<>(SearchTemplateAction.INSTANCE, TransportSearchTemplateAction.class), new ActionHandler<>(MultiSearchTemplateAction.INSTANCE, TransportMultiSearchTemplateAction.class), - new ActionHandler<>(DeleteStoredSearchTemplateAction.INSTANCE, TransportDeleteStoredSearchTemplateAction.class), - new ActionHandler<>(GetStoredSearchTemplateAction.INSTANCE, TransportGetStoredSearchTemplateAction.class), - new ActionHandler<>(PutStoredSearchTemplateAction.INSTANCE, TransportPutStoredSearchTemplateAction.class)); + new ActionHandler<>(DeleteStoredTemplateAction.INSTANCE, TransportDeleteStoredTemplateAction.class), + new ActionHandler<>(GetStoredTemplateAction.INSTANCE, TransportGetStoredTemplateAction.class), + new ActionHandler<>(PutStoredTemplateAction.INSTANCE, TransportPutStoredSearchTemplateAction.class)); } @Override @@ -78,9 +78,9 @@ public List getRestHandlers(Settings settings, RestController restC return Arrays.asList( new RestSearchTemplateAction(settings, restController), new RestMultiSearchTemplateAction(settings, restController), - new RestGetStoredSearchTemplateAction(settings, restController), - new RestPutStoredSearchTemplateAction(settings, restController), - new RestDeleteStoredSearchTemplateAction(settings, restController), + new RestGetStoredTemplateAction(settings, restController), + new RestPutStoredTemplateAction(settings, restController), + new RestDeleteStoredTemplateAction(settings, restController), new RestRenderSearchTemplateAction(settings, restController)); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateAction.java similarity index 62% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateAction.java index d8c977c40bc79..6b161086d0b46 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateAction.java @@ -22,26 +22,26 @@ import org.elasticsearch.action.Action; import org.elasticsearch.client.ElasticsearchClient; -public class DeleteStoredSearchTemplateAction extends Action< - DeleteStoredSearchTemplateRequest, - DeleteStoredSearchTemplateResponse, - DeleteStoredSearchTemplateRequestBuilder> { +public class DeleteStoredTemplateAction extends Action< + DeleteStoredTemplateRequest, + DeleteStoredTemplateResponse, + DeleteStoredTemplateRequestBuilder> { - public static final DeleteStoredSearchTemplateAction INSTANCE = - new DeleteStoredSearchTemplateAction(); + public static final DeleteStoredTemplateAction INSTANCE = + new DeleteStoredTemplateAction(); public static final String NAME = "cluster:admin/search/template/delete"; - private DeleteStoredSearchTemplateAction() { + private DeleteStoredTemplateAction() { super(NAME); } @Override - public DeleteStoredSearchTemplateResponse newResponse() { - return new DeleteStoredSearchTemplateResponse(); + public DeleteStoredTemplateResponse newResponse() { + return new DeleteStoredTemplateResponse(); } @Override - public DeleteStoredSearchTemplateRequestBuilder newRequestBuilder(ElasticsearchClient client) { - return new DeleteStoredSearchTemplateRequestBuilder(client, this); + public DeleteStoredTemplateRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new DeleteStoredTemplateRequestBuilder(client, this); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateRequest.java similarity index 89% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateRequest.java index 3a2d9e43fdb7a..449862086fa3e 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateRequest.java @@ -28,19 +28,19 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public class DeleteStoredSearchTemplateRequest extends AcknowledgedRequest< - DeleteStoredSearchTemplateRequest> { +public class DeleteStoredTemplateRequest extends AcknowledgedRequest< + DeleteStoredTemplateRequest> { private String id; - DeleteStoredSearchTemplateRequest() { + DeleteStoredTemplateRequest() { } - public DeleteStoredSearchTemplateRequest(String id) { + public DeleteStoredTemplateRequest(String id) { this.id = id; } - public DeleteStoredSearchTemplateRequest id(String id) { + public DeleteStoredTemplateRequest id(String id) { this.id = id; return this; } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequestBuilder.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateRequestBuilder.java similarity index 67% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequestBuilder.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateRequestBuilder.java index ab0c563b60aee..8103891df2fb3 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateRequestBuilder.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateRequestBuilder.java @@ -22,17 +22,17 @@ import org.elasticsearch.action.support.master.AcknowledgedRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; -public class DeleteStoredSearchTemplateRequestBuilder extends AcknowledgedRequestBuilder< - DeleteStoredSearchTemplateRequest, - DeleteStoredSearchTemplateResponse, - DeleteStoredSearchTemplateRequestBuilder> { +public class DeleteStoredTemplateRequestBuilder extends AcknowledgedRequestBuilder< + DeleteStoredTemplateRequest, + DeleteStoredTemplateResponse, + DeleteStoredTemplateRequestBuilder> { - public DeleteStoredSearchTemplateRequestBuilder(ElasticsearchClient client, - DeleteStoredSearchTemplateAction action) { - super(client, action, new DeleteStoredSearchTemplateRequest()); + public DeleteStoredTemplateRequestBuilder(ElasticsearchClient client, + DeleteStoredTemplateAction action) { + super(client, action, new DeleteStoredTemplateRequest()); } - public DeleteStoredSearchTemplateRequestBuilder setId(String id) { + public DeleteStoredTemplateRequestBuilder setId(String id) { request.id(id); return this; } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateResponse.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateResponse.java similarity index 88% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateResponse.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateResponse.java index a28f2b60f5e22..aab2172b2e735 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateResponse.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredTemplateResponse.java @@ -25,11 +25,12 @@ import java.io.IOException; -public class PutStoredSearchTemplateResponse extends AcknowledgedResponse { - PutStoredSearchTemplateResponse() { +public class DeleteStoredTemplateResponse extends AcknowledgedResponse { + + DeleteStoredTemplateResponse() { } - public PutStoredSearchTemplateResponse(boolean acknowledged) { + public DeleteStoredTemplateResponse(boolean acknowledged) { super(acknowledged); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateAction.java similarity index 63% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateAction.java index 52ad8adcca410..6ae26a4d97b03 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateAction.java @@ -22,27 +22,27 @@ import org.elasticsearch.action.Action; import org.elasticsearch.client.ElasticsearchClient; -public class GetStoredSearchTemplateAction extends Action< - GetStoredSearchTemplateRequest, - GetStoredSearchTemplateResponse, - GetStoredSearchTemplateRequestBuilder> { +public class GetStoredTemplateAction extends Action< + GetStoredTemplateRequest, + GetStoredTemplateResponse, + GetStoredTemplateRequestBuilder> { - public static final GetStoredSearchTemplateAction INSTANCE = - new GetStoredSearchTemplateAction(); + public static final GetStoredTemplateAction INSTANCE = + new GetStoredTemplateAction(); public static final String NAME = "cluster:admin/search/template/get"; - private GetStoredSearchTemplateAction() { + private GetStoredTemplateAction() { super(NAME); } @Override - public GetStoredSearchTemplateResponse newResponse() { - return new GetStoredSearchTemplateResponse(); + public GetStoredTemplateResponse newResponse() { + return new GetStoredTemplateResponse(); } @Override - public GetStoredSearchTemplateRequestBuilder newRequestBuilder(ElasticsearchClient client) { - return new GetStoredSearchTemplateRequestBuilder(client, this); + public GetStoredTemplateRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new GetStoredTemplateRequestBuilder(client, this); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateRequest.java similarity index 89% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateRequest.java index d96123cb6e937..1ae2a68d85a01 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateRequest.java @@ -28,19 +28,19 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public class GetStoredSearchTemplateRequest extends MasterNodeReadRequest< - GetStoredSearchTemplateRequest> { +public class GetStoredTemplateRequest extends MasterNodeReadRequest< + GetStoredTemplateRequest> { protected String id; - GetStoredSearchTemplateRequest() { + GetStoredTemplateRequest() { } - public GetStoredSearchTemplateRequest(String id) { + public GetStoredTemplateRequest(String id) { this.id = id; } - public GetStoredSearchTemplateRequest id(String id) { + public GetStoredTemplateRequest id(String id) { this.id = id; return this; } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequestBuilder.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateRequestBuilder.java similarity index 67% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequestBuilder.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateRequestBuilder.java index 41f9929599987..61436840fa5c0 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateRequestBuilder.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateRequestBuilder.java @@ -22,17 +22,17 @@ import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; -public class GetStoredSearchTemplateRequestBuilder extends MasterNodeReadOperationRequestBuilder< - GetStoredSearchTemplateRequest, - GetStoredSearchTemplateResponse, - GetStoredSearchTemplateRequestBuilder> { +public class GetStoredTemplateRequestBuilder extends MasterNodeReadOperationRequestBuilder< + GetStoredTemplateRequest, + GetStoredTemplateResponse, + GetStoredTemplateRequestBuilder> { - public GetStoredSearchTemplateRequestBuilder(ElasticsearchClient client, - GetStoredSearchTemplateAction action) { - super(client, action, new GetStoredSearchTemplateRequest()); + public GetStoredTemplateRequestBuilder(ElasticsearchClient client, + GetStoredTemplateAction action) { + super(client, action, new GetStoredTemplateRequest()); } - public GetStoredSearchTemplateRequestBuilder setId(String id) { + public GetStoredTemplateRequestBuilder setId(String id) { request.id(id); return this; } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateResponse.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateResponse.java similarity index 91% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateResponse.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateResponse.java index 284935faa1a95..9a60c60fd76c3 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredSearchTemplateResponse.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/GetStoredTemplateResponse.java @@ -28,14 +28,14 @@ import java.io.IOException; -public class GetStoredSearchTemplateResponse extends ActionResponse implements ToXContent { +public class GetStoredTemplateResponse extends ActionResponse implements ToXContent { private StoredScriptSource source; - GetStoredSearchTemplateResponse() { + GetStoredTemplateResponse() { } - GetStoredSearchTemplateResponse(StoredScriptSource source) { + GetStoredTemplateResponse(StoredScriptSource source) { this.source = source; } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateAction.java similarity index 63% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateAction.java index f9b4c70e40da3..bef0390ed681e 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateAction.java @@ -22,26 +22,26 @@ import org.elasticsearch.action.Action; import org.elasticsearch.client.ElasticsearchClient; -public class PutStoredSearchTemplateAction extends Action< - PutStoredSearchTemplateRequest, - PutStoredSearchTemplateResponse, - PutStoredSearchTemplateRequestBuilder> { +public class PutStoredTemplateAction extends Action< + PutStoredTemplateRequest, + PutStoredTemplateResponse, + PutStoredTemplateRequestBuilder> { - public static final PutStoredSearchTemplateAction INSTANCE = - new PutStoredSearchTemplateAction(); + public static final PutStoredTemplateAction INSTANCE = + new PutStoredTemplateAction(); public static final String NAME = "cluster:admin/search/template/put"; - private PutStoredSearchTemplateAction() { + private PutStoredTemplateAction() { super(NAME); } @Override - public PutStoredSearchTemplateResponse newResponse() { - return new PutStoredSearchTemplateResponse(); + public PutStoredTemplateResponse newResponse() { + return new PutStoredTemplateResponse(); } @Override - public PutStoredSearchTemplateRequestBuilder newRequestBuilder(ElasticsearchClient client) { - return new PutStoredSearchTemplateRequestBuilder(client, this); + public PutStoredTemplateRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new PutStoredTemplateRequestBuilder(client, this); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateRequest.java similarity index 90% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequest.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateRequest.java index 8f4904f922b91..f5d9f75e4376b 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateRequest.java @@ -32,28 +32,28 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public class PutStoredSearchTemplateRequest extends AcknowledgedRequest< - PutStoredSearchTemplateRequest> { +public class PutStoredTemplateRequest extends AcknowledgedRequest< + PutStoredTemplateRequest> { private String id; private BytesReference content; private XContentType xContentType; - public PutStoredSearchTemplateRequest() { + public PutStoredTemplateRequest() { } - public PutStoredSearchTemplateRequest(String id, BytesReference content, + public PutStoredTemplateRequest(String id, BytesReference content, XContentType xContentType) { this.id = id; this.content = content; this.xContentType = Objects.requireNonNull(xContentType); } - public PutStoredSearchTemplateRequest id(String id) { + public PutStoredTemplateRequest id(String id) { this.id = id; return this; } - public PutStoredSearchTemplateRequest content(BytesReference content, + public PutStoredTemplateRequest content(BytesReference content, XContentType xContentType) { this.content = content; this.xContentType = xContentType; diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequestBuilder.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateRequestBuilder.java similarity index 68% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequestBuilder.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateRequestBuilder.java index b8982900bc6df..91cbb2399d6b2 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredSearchTemplateRequestBuilder.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateRequestBuilder.java @@ -24,22 +24,22 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.xcontent.XContentType; -public class PutStoredSearchTemplateRequestBuilder extends AcknowledgedRequestBuilder< - PutStoredSearchTemplateRequest, - PutStoredSearchTemplateResponse, - PutStoredSearchTemplateRequestBuilder> { +public class PutStoredTemplateRequestBuilder extends AcknowledgedRequestBuilder< + PutStoredTemplateRequest, + PutStoredTemplateResponse, + PutStoredTemplateRequestBuilder> { - public PutStoredSearchTemplateRequestBuilder(ElasticsearchClient client, - PutStoredSearchTemplateAction action) { - super(client, action, new PutStoredSearchTemplateRequest()); + public PutStoredTemplateRequestBuilder(ElasticsearchClient client, + PutStoredTemplateAction action) { + super(client, action, new PutStoredTemplateRequest()); } - public PutStoredSearchTemplateRequestBuilder setId(String id) { + public PutStoredTemplateRequestBuilder setId(String id) { request.id(id); return this; } - public PutStoredSearchTemplateRequestBuilder setContent(BytesReference content, + public PutStoredTemplateRequestBuilder setContent(BytesReference content, XContentType xContentType) { request.content(content, xContentType); return this; diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateResponse.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateResponse.java similarity index 87% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateResponse.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateResponse.java index 823b86c387775..4415227c7744d 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/DeleteStoredSearchTemplateResponse.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/PutStoredTemplateResponse.java @@ -25,12 +25,11 @@ import java.io.IOException; -public class DeleteStoredSearchTemplateResponse extends AcknowledgedResponse { - - DeleteStoredSearchTemplateResponse() { +public class PutStoredTemplateResponse extends AcknowledgedResponse { + PutStoredTemplateResponse() { } - public DeleteStoredSearchTemplateResponse(boolean acknowledged) { + public PutStoredTemplateResponse(boolean acknowledged) { super(acknowledged); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredTemplateAction.java similarity index 80% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredTemplateAction.java index 2d4926cee8a68..2621ee8a2e5cf 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestDeleteStoredTemplateAction.java @@ -29,9 +29,9 @@ import static org.elasticsearch.rest.RestRequest.Method.DELETE; -public class RestDeleteStoredSearchTemplateAction extends BaseRestHandler { +public class RestDeleteStoredTemplateAction extends BaseRestHandler { - public RestDeleteStoredSearchTemplateAction(Settings settings, RestController controller) { + public RestDeleteStoredTemplateAction(Settings settings, RestController controller) { super(settings); controller.registerHandler(DELETE, "/_search/template/{id}", this); @@ -42,8 +42,8 @@ public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client throws IOException { String id = request.param("id"); - DeleteStoredSearchTemplateRequest deleteRequest = new DeleteStoredSearchTemplateRequest(id); - return channel -> client.execute(DeleteStoredSearchTemplateAction.INSTANCE, deleteRequest, + DeleteStoredTemplateRequest deleteRequest = new DeleteStoredTemplateRequest(id); + return channel -> client.execute(DeleteStoredTemplateAction.INSTANCE, deleteRequest, new AcknowledgedRestListener<>(channel)); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredTemplateAction.java similarity index 85% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredTemplateAction.java index bb0e0b5352283..f8442cf40272c 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestGetStoredTemplateAction.java @@ -36,13 +36,13 @@ import static org.elasticsearch.rest.RestRequest.Method.GET; -public class RestGetStoredSearchTemplateAction extends BaseRestHandler { +public class RestGetStoredTemplateAction extends BaseRestHandler { public static final ParseField _ID_PARSE_FIELD = new ParseField("_id"); public static final ParseField FOUND_PARSE_FIELD = new ParseField("found"); - public RestGetStoredSearchTemplateAction(Settings settings, RestController controller) { + public RestGetStoredTemplateAction(Settings settings, RestController controller) { super(settings); controller.registerHandler(GET, "/_search/template/{id}", this); @@ -53,12 +53,12 @@ public RestChannelConsumer prepareRequest(final RestRequest request, NodeClient throws IOException { String id = request.param("id"); - GetStoredSearchTemplateRequest getRequest = new GetStoredSearchTemplateRequest(id); + GetStoredTemplateRequest getRequest = new GetStoredTemplateRequest(id); - return channel -> client.execute(GetStoredSearchTemplateAction.INSTANCE, getRequest, - new RestBuilderListener(channel) { + return channel -> client.execute(GetStoredTemplateAction.INSTANCE, getRequest, + new RestBuilderListener(channel) { @Override - public RestResponse buildResponse(GetStoredSearchTemplateResponse response, + public RestResponse buildResponse(GetStoredTemplateResponse response, XContentBuilder builder) throws Exception { builder.startObject(); builder.field("_id", id); diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredTemplateAction.java similarity index 84% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredTemplateAction.java index 85a36bc4eafc1..6fb7a03d548e2 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/RestPutStoredTemplateAction.java @@ -31,9 +31,9 @@ import static org.elasticsearch.rest.RestRequest.Method.POST; import static org.elasticsearch.rest.RestRequest.Method.PUT; -public class RestPutStoredSearchTemplateAction extends BaseRestHandler { +public class RestPutStoredTemplateAction extends BaseRestHandler { - public RestPutStoredSearchTemplateAction(Settings settings, RestController controller) { + public RestPutStoredTemplateAction(Settings settings, RestController controller) { super(settings); controller.registerHandler(POST, "/_search/template/{id}", this); @@ -46,9 +46,9 @@ public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client String id = request.param("id"); BytesReference content = request.content(); - PutStoredSearchTemplateRequest put = new PutStoredSearchTemplateRequest(id, content, + PutStoredTemplateRequest put = new PutStoredTemplateRequest(id, content, request.getXContentType()); - return channel -> client.execute(PutStoredSearchTemplateAction.INSTANCE, put, + return channel -> client.execute(PutStoredTemplateAction.INSTANCE, put, new AcknowledgedRestListener<>(channel)); } } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredTemplateAction.java similarity index 75% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredTemplateAction.java index 0a2d33c4c1f0b..d333a4c8205c2 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportDeleteStoredTemplateAction.java @@ -36,19 +36,19 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -public class TransportDeleteStoredSearchTemplateAction extends TransportMasterNodeAction< - DeleteStoredSearchTemplateRequest, DeleteStoredSearchTemplateResponse> { +public class TransportDeleteStoredTemplateAction extends TransportMasterNodeAction< + DeleteStoredTemplateRequest, DeleteStoredTemplateResponse> { private final TemplateService templateService; @Inject - public TransportDeleteStoredSearchTemplateAction(Settings settings, + public TransportDeleteStoredTemplateAction(Settings settings, TransportService transportService, ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, TemplateService templateService) { - super(settings, DeleteStoredSearchTemplateAction.NAME, transportService, clusterService, + super(settings, DeleteStoredTemplateAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, - DeleteStoredSearchTemplateRequest::new); + DeleteStoredTemplateRequest::new); this.templateService = templateService; } @@ -58,19 +58,19 @@ protected String executor() { } @Override - protected DeleteStoredSearchTemplateResponse newResponse() { - return new DeleteStoredSearchTemplateResponse(); + protected DeleteStoredTemplateResponse newResponse() { + return new DeleteStoredTemplateResponse(); } @Override - protected void masterOperation(DeleteStoredSearchTemplateRequest request, ClusterState state, - ActionListener listener) throws Exception { + protected void masterOperation(DeleteStoredTemplateRequest request, ClusterState state, + ActionListener listener) throws Exception { clusterService.submitStateUpdateTask("delete-search-template-" + request.id(), - new AckedClusterStateUpdateTask(request, + new AckedClusterStateUpdateTask(request, listener) { @Override - protected DeleteStoredSearchTemplateResponse newResponse(boolean acknowledged) { - return new DeleteStoredSearchTemplateResponse(acknowledged); + protected DeleteStoredTemplateResponse newResponse(boolean acknowledged) { + return new DeleteStoredTemplateResponse(acknowledged); } @Override @@ -86,7 +86,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { } @Override - protected ClusterBlockException checkBlock(DeleteStoredSearchTemplateRequest request, + protected ClusterBlockException checkBlock(DeleteStoredTemplateRequest request, ClusterState state) { return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredTemplateAction.java similarity index 75% rename from modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredSearchTemplateAction.java rename to modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredTemplateAction.java index 2a85033cb9560..4a4589d496fd7 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportGetStoredTemplateAction.java @@ -34,19 +34,19 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -public class TransportGetStoredSearchTemplateAction extends TransportMasterNodeReadAction< - GetStoredSearchTemplateRequest, GetStoredSearchTemplateResponse> { +public class TransportGetStoredTemplateAction extends TransportMasterNodeReadAction< + GetStoredTemplateRequest, GetStoredTemplateResponse> { private final TemplateService templateService; @Inject - public TransportGetStoredSearchTemplateAction(Settings settings, + public TransportGetStoredTemplateAction(Settings settings, TransportService transportService, ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, TemplateService templateService) { - super(settings, GetStoredSearchTemplateAction.NAME, transportService, clusterService, + super(settings, GetStoredTemplateAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, - GetStoredSearchTemplateRequest::new); + GetStoredTemplateRequest::new); this.templateService = templateService; } @@ -56,17 +56,17 @@ protected String executor() { } @Override - protected GetStoredSearchTemplateResponse newResponse() { - return new GetStoredSearchTemplateResponse(); + protected GetStoredTemplateResponse newResponse() { + return new GetStoredTemplateResponse(); } @Override - protected void masterOperation(GetStoredSearchTemplateRequest request, ClusterState state, - ActionListener listener) throws Exception { + protected void masterOperation(GetStoredTemplateRequest request, ClusterState state, + ActionListener listener) throws Exception { ScriptMetaData scriptMetadata = state.metaData().custom(ScriptMetaData.TYPE); if (scriptMetadata != null) { - listener.onResponse(new GetStoredSearchTemplateResponse(scriptMetadata + listener.onResponse(new GetStoredTemplateResponse(scriptMetadata .getStoredScript(request.id(), templateService.getTemplateLanguage()))); } else { listener.onResponse(null); @@ -74,7 +74,7 @@ protected void masterOperation(GetStoredSearchTemplateRequest request, ClusterSt } @Override - protected ClusterBlockException checkBlock(GetStoredSearchTemplateRequest request, + protected ClusterBlockException checkBlock(GetStoredTemplateRequest request, ClusterState state) { return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java index befb03175ee16..c9218e95001a0 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/stored/TransportPutStoredSearchTemplateAction.java @@ -40,7 +40,7 @@ import org.elasticsearch.transport.TransportService; public class TransportPutStoredSearchTemplateAction extends TransportMasterNodeAction< - PutStoredSearchTemplateRequest, PutStoredSearchTemplateResponse> { + PutStoredTemplateRequest, PutStoredTemplateResponse> { private final TemplateService templateService; private final int maxScriptSizeInBytes; @@ -50,9 +50,9 @@ public TransportPutStoredSearchTemplateAction(Settings settings, TransportService transportService, ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, TemplateService templateService) { - super(settings, PutStoredSearchTemplateAction.NAME, transportService, clusterService, + super(settings, PutStoredTemplateAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, - PutStoredSearchTemplateRequest::new); + PutStoredTemplateRequest::new); this.templateService = templateService; maxScriptSizeInBytes = ScriptService.SCRIPT_MAX_SIZE_IN_BYTES.get(settings); } @@ -63,13 +63,13 @@ protected String executor() { } @Override - protected PutStoredSearchTemplateResponse newResponse() { - return new PutStoredSearchTemplateResponse(); + protected PutStoredTemplateResponse newResponse() { + return new PutStoredTemplateResponse(); } @Override - protected void masterOperation(PutStoredSearchTemplateRequest request, ClusterState state, - ActionListener listener) throws Exception { + protected void masterOperation(PutStoredTemplateRequest request, ClusterState state, + ActionListener listener) throws Exception { if (request.content().length() > maxScriptSizeInBytes) { throw new IllegalArgumentException("exceeded max allowed stored script size in bytes [" + maxScriptSizeInBytes + "] with size [" + request.content().length() @@ -88,12 +88,12 @@ protected void masterOperation(PutStoredSearchTemplateRequest request, ClusterSt } clusterService.submitStateUpdateTask("put-search-template-" + request.id(), - new AckedClusterStateUpdateTask(request, + new AckedClusterStateUpdateTask(request, listener) { @Override - protected PutStoredSearchTemplateResponse newResponse(boolean acknowledged) { - return new PutStoredSearchTemplateResponse(acknowledged); + protected PutStoredTemplateResponse newResponse(boolean acknowledged) { + return new PutStoredTemplateResponse(acknowledged); } @Override @@ -109,7 +109,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { } @Override - protected ClusterBlockException checkBlock(PutStoredSearchTemplateRequest request, + protected ClusterBlockException checkBlock(PutStoredTemplateRequest request, ClusterState state) { return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); } diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java index 3de0f39e7863c..b5411a5e768e2 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java @@ -28,10 +28,10 @@ import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.ScriptType; -import org.elasticsearch.script.mustache.stored.DeleteStoredSearchTemplateAction; -import org.elasticsearch.script.mustache.stored.GetStoredSearchTemplateAction; -import org.elasticsearch.script.mustache.stored.GetStoredSearchTemplateResponse; -import org.elasticsearch.script.mustache.stored.PutStoredSearchTemplateAction; +import org.elasticsearch.script.mustache.stored.DeleteStoredTemplateAction; +import org.elasticsearch.script.mustache.stored.GetStoredTemplateAction; +import org.elasticsearch.script.mustache.stored.GetStoredTemplateResponse; +import org.elasticsearch.script.mustache.stored.PutStoredTemplateAction; import org.elasticsearch.test.ESSingleNodeTestCase; import org.junit.Before; @@ -154,7 +154,7 @@ public void testTemplateQueryAsEscapedStringWithConditionalClauseAtEnd() throws } public void testIndexedTemplateClient() throws Exception { - assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) + assertAcked(PutStoredTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("testTemplate") .setContent(new BytesArray("{" + "\"template\":{" + @@ -166,7 +166,7 @@ public void testIndexedTemplateClient() throws Exception { "}"), XContentType.JSON)); - assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) + assertAcked(PutStoredTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("testTemplate").setContent(new BytesArray("{" + "\"template\":{" + " \"query\":{" + @@ -176,7 +176,7 @@ public void testIndexedTemplateClient() throws Exception { "}" + "}"), XContentType.JSON)); - GetStoredSearchTemplateResponse getResponse = GetStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) + GetStoredTemplateResponse getResponse = GetStoredTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("testTemplate").get(); assertNotNull(getResponse.getSource()); @@ -198,9 +198,9 @@ public void testIndexedTemplateClient() throws Exception { .get(); assertHitCount(searchResponse.getResponse(), 4); - assertAcked(DeleteStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()).setId("testTemplate")); + assertAcked(DeleteStoredTemplateAction.INSTANCE.newRequestBuilder(client()).setId("testTemplate")); - getResponse = GetStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()).setId("testTemplate").get(); + getResponse = GetStoredTemplateAction.INSTANCE.newRequestBuilder(client()).setId("testTemplate").get(); assertNull(getResponse.getSource()); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new SearchTemplateRequestBuilder(client()) @@ -211,7 +211,7 @@ public void testIndexedTemplateClient() throws Exception { } public void testIndexedTemplate() throws Exception { - assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) + assertAcked(PutStoredTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("1a") .setContent(new BytesArray("{" + "\"template\":{" + @@ -223,7 +223,7 @@ public void testIndexedTemplate() throws Exception { "}" ), XContentType.JSON) ); - assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) + assertAcked(PutStoredTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("2") .setContent(new BytesArray("{" + "\"template\":{" + @@ -234,7 +234,7 @@ public void testIndexedTemplate() throws Exception { "}" + "}"), XContentType.JSON) ); - assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) + assertAcked(PutStoredTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("3") .setContent(new BytesArray("{" + "\"template\":{" + @@ -309,7 +309,7 @@ public void testIndexedTemplateOverwrite() throws Exception { int iterations = randomIntBetween(2, 11); for (int i = 1; i < iterations; i++) { - assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) + assertAcked(PutStoredTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("git01") .setContent(new BytesArray("{\"template\":{\"query\": {\"match\": {\"searchtext\": {\"query\": \"{{P_Keyword1}}\"," + "\"type\": \"ooophrase_prefix\"}}}}}"), XContentType.JSON)); @@ -328,7 +328,7 @@ public void testIndexedTemplateOverwrite() throws Exception { assertThat(e.getMessage(), containsString("[match] query does not support type ooophrase_prefix")); assertWarnings("Deprecated field [type] used, replaced by [match_phrase and match_phrase_prefix query]"); - assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) + assertAcked(PutStoredTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("git01") .setContent(new BytesArray("{\"query\": {\"match\": {\"searchtext\": {\"query\": \"{{P_Keyword1}}\"," + "\"type\": \"phrase_prefix\"}}}}"), XContentType.JSON)); @@ -344,7 +344,7 @@ public void testIndexedTemplateOverwrite() throws Exception { public void testIndexedTemplateWithArray() throws Exception { String multiQuery = "{\"query\":{\"terms\":{\"theField\":[\"{{#fieldParam}}\",\"{{.}}\",\"{{/fieldParam}}\"]}}}"; - assertAcked(PutStoredSearchTemplateAction.INSTANCE.newRequestBuilder(client()) + assertAcked(PutStoredTemplateAction.INSTANCE.newRequestBuilder(client()) .setId("4") .setContent(jsonBuilder().startObject().field("template", multiQuery).endObject().bytes(), XContentType.JSON)); BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); From e06738efc8ad0bad30cf0fd5f96e3620f8ba2cd9 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 24 Mar 2017 17:21:45 -0400 Subject: [PATCH 14/37] Migration notes for template/script split --- .../migration/migrate_6_0/scripting.asciidoc | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/reference/migration/migrate_6_0/scripting.asciidoc b/docs/reference/migration/migrate_6_0/scripting.asciidoc index 6740200922327..8b8defc9eea8e 100644 --- a/docs/reference/migration/migrate_6_0/scripting.asciidoc +++ b/docs/reference/migration/migrate_6_0/scripting.asciidoc @@ -12,3 +12,15 @@ Use painless instead. milliseconds since epoch as a `long`. The same is true for `doc.some_date_field[some_number]`. Use `doc.some_date_field.value.millis` to fetch the milliseconds since epoch if you need it. + +==== Scripts and Templates are separate things now + +Internally we've split scripts and templates apart. This shouldn't matter over +the REST interface except during the rolling upgrade from 5.x to 6.0. During +that upgrade requests to REST APIs for templates aren't going to work. + +Transport client users will need to use some new actions in the lang-mustache +module to work with templates: +* `PutStoredTemplateAction` +* `GetStoredTemplateAction` +* `DeleteStoredTemplateAction` From 42fb904d36150701e797c6895449436b5ac77de1 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 24 Mar 2017 17:48:51 -0400 Subject: [PATCH 15/37] Drop duplicate log message --- .../java/org/elasticsearch/script/CachingCompiler.java | 1 - docs/reference/migration/migrate_6_0/scripting.asciidoc | 7 +++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index f1cc579c8c5a1..7d44ecdd171ff 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -313,7 +313,6 @@ public void onFileInit(Path file) { CacheKeyT cacheKey = cacheKeyForFile(scriptNameExt.v1(), scriptNameExt.v2()); if (cacheKey == null) { - logger.warn("No script engine found for [{}]", scriptNameExt.v2()); return; } try { diff --git a/docs/reference/migration/migrate_6_0/scripting.asciidoc b/docs/reference/migration/migrate_6_0/scripting.asciidoc index 8b8defc9eea8e..c7973aea786de 100644 --- a/docs/reference/migration/migrate_6_0/scripting.asciidoc +++ b/docs/reference/migration/migrate_6_0/scripting.asciidoc @@ -16,8 +16,11 @@ fetch the milliseconds since epoch if you need it. ==== Scripts and Templates are separate things now Internally we've split scripts and templates apart. This shouldn't matter over -the REST interface except during the rolling upgrade from 5.x to 6.0. During -that upgrade requests to REST APIs for templates aren't going to work. +the REST interface except for two things: +* During the rolling upgrade from 5.x to 6.0 the REST APIs for templates aren't +going to work properly most of the time. Don't use them. +* You can no longer PUT/GET/DELETE templates using the script API and you can +no longer PUT/GET/DELETE scripts using the template API. Transport client users will need to use some new actions in the lang-mustache module to work with templates: From 207e41752b8655062c859c2a14714ac6371e63b6 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 24 Mar 2017 17:50:32 -0400 Subject: [PATCH 16/37] Remove leftover --- .../script/mustache/MustacheScriptEngineService.java | 1 - 1 file changed, 1 deletion(-) diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java index 7d7368b577867..48b652d0fefc4 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheScriptEngineService.java @@ -107,7 +107,6 @@ public String getExtension() { @Override public ExecutableScript executable(CompiledScript compiledScript, @Nullable Map vars) { - ESLoggerFactory.getLogger(TemplateService.class).warn("ASDFASDF toexec [{}] [{}]", compiledScript, vars); return new MustacheExecutableScript(compiledScript, vars); } From 4d0b2f2cc3ab82d1e5bece6bc7d34527d61673ff Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sat, 25 Mar 2017 11:01:46 -0400 Subject: [PATCH 17/37] Implement contentType in TemplateService --- .../index/query/QueryRewriteContext.java | 5 +- .../ingest/InternalTemplateService.java | 2 +- .../elasticsearch/script/TemplateService.java | 87 ++++++++++++++---- .../phrase/PhraseSuggestionBuilder.java | 2 +- .../script/TemplateServiceTests.java | 16 ++-- .../mustache/MustacheTemplateService.java | 41 --------- .../TransportSearchTemplateAction.java | 12 +-- .../script/mustache/ContentTypeTests.java | 91 +++++++++++++++++++ .../mustache/CustomMustacheFactoryTests.java | 4 +- 9 files changed, 181 insertions(+), 79 deletions(-) delete mode 100644 modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheTemplateService.java create mode 100644 modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/ContentTypeTests.java diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java index e5177e7e368ac..c834b5b4ac272 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryRewriteContext.java @@ -105,9 +105,10 @@ public long nowInMillis() { return nowInMillis.getAsLong(); } - public BytesReference getTemplateBytes(Script template) { // TODO remove this and use getTemplateService directly + public BytesReference getTemplateBytes(Script template) { return templateService - .template(template.getIdOrCode(), template.getType(), ScriptContext.Standard.SEARCH) + .template(template.getIdOrCode(), template.getType(), + ScriptContext.Standard.SEARCH, null) .apply(template.getParams()); } diff --git a/core/src/main/java/org/elasticsearch/ingest/InternalTemplateService.java b/core/src/main/java/org/elasticsearch/ingest/InternalTemplateService.java index 953212c4b9bea..a3b51ca703c7e 100644 --- a/core/src/main/java/org/elasticsearch/ingest/InternalTemplateService.java +++ b/core/src/main/java/org/elasticsearch/ingest/InternalTemplateService.java @@ -40,7 +40,7 @@ public Template compile(String template) { int mustacheEnd = template.indexOf("}}"); if (mustacheStart != -1 && mustacheEnd != -1 && mustacheStart < mustacheEnd) { Function, BytesReference> compiled = templateService.template( - template, ScriptType.INLINE, ScriptContext.Standard.INGEST); + template, ScriptType.INLINE, ScriptContext.Standard.INGEST, null); return new Template() { @Override public String execute(Map model) { diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index 614671db191f7..28072a2025164 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -24,6 +24,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.ESLoggerFactory; @@ -38,15 +39,20 @@ import java.util.function.Function; import static java.util.Collections.emptyMap; +import static java.util.Collections.singletonMap; +/** + * Source of templates (think Mustache or StringTemplate). + */ public class TemplateService implements ClusterStateListener { public interface Backend extends ScriptEngineService {} // TODO customize this for templates + private static final String DEFAULT_CONTENT_TYPE = "text/plain"; private static final Logger logger = ESLoggerFactory.getLogger(TemplateService.class); private static final DeprecationLogger deprecationLogger = new DeprecationLogger(logger); private final Backend backend; private final ScriptPermits scriptPermits; - private final CachingCompiler compiler; + private final CachingCompiler compiler; public TemplateService(Settings settings, Environment env, ResourceWatcherService resourceWatcherService, Backend backend, @@ -56,26 +62,33 @@ public TemplateService(Settings settings, Environment env, this.backend = backend; this.scriptPermits = new ScriptPermits(settings, scriptSettings, scriptContextRegistry); - this.compiler = new CachingCompiler(settings, env, resourceWatcherService, + this.compiler = new CachingCompiler(settings, env, resourceWatcherService, scriptMetrics, "template") { @Override - protected String cacheKeyForFile(String baseName, String extension) { + protected CacheKey cacheKeyForFile(String baseName, String extension) { if (false == backend.getType().equals(extension)) { /* For backwards compatibility templates are in the scripts directory and we * must ignore everything but templates. */ return null; } - return baseName; + return new CacheKey(baseName, DEFAULT_CONTENT_TYPE); } @Override - protected String cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { - return scriptMetadata.getCode(); + protected CacheKey cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { + String contentType = DEFAULT_CONTENT_TYPE; + if (scriptMetadata.getOptions() != null) { + contentType = scriptMetadata.getOptions().get(Script.CONTENT_TYPE_OPTION); + if (contentType == null) { + contentType = DEFAULT_CONTENT_TYPE; + } + } + return new CacheKey(scriptMetadata.getCode(), contentType); } @Override protected StoredScriptSource lookupStoredScript(ClusterState clusterState, - String cacheKey) { + CacheKey cacheKey) { if (clusterState == null) { return null; } @@ -88,7 +101,9 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, return null; } - String id = cacheKey; + /* This process throws away contentType which is maybe bad, but it is the behavior + * that we've had in templates for a long time. */ + String id = cacheKey.idOrCode; // search template requests can possibly pass in the entire path instead // of just an id for looking up a stored script, so we parse the path and // check for appropriate errors @@ -96,7 +111,6 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, if (path.length == 3) { id = path[2]; - deprecationLogger.deprecated("use of [{}] for looking up stored " + "scripts/templates has been deprecated, use only [{}] instead", cacheKey, id); @@ -109,7 +123,7 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, } @Override - protected boolean areAnyScriptContextsEnabled(String cacheKey, ScriptType scriptType) { + protected boolean areAnyScriptContextsEnabled(CacheKey cacheKey, ScriptType scriptType) { for (ScriptContext scriptContext : scriptContextRegistry.scriptContexts()) { if (scriptPermits.checkContextPermissions(backend.getType(), scriptType, scriptContext)) { @@ -120,7 +134,7 @@ protected boolean areAnyScriptContextsEnabled(String cacheKey, ScriptType script } @Override - protected void checkContextPermissions(String cacheKey, ScriptType scriptType, + protected void checkContextPermissions(CacheKey cacheKey, ScriptType scriptType, ScriptContext scriptContext) { if (scriptPermits.checkContextPermissions(backend.getType(), scriptType, scriptContext) == false) { @@ -135,13 +149,15 @@ protected void checkCompilationLimit() { } @Override - protected CompiledScript compile(ScriptType scriptType, String cacheKey) { - Object compiled = backend.compile(null, cacheKey, emptyMap()); - return new CompiledScript(scriptType, cacheKey, backend.getType(), compiled); + protected CompiledScript compile(ScriptType scriptType, CacheKey cacheKey) { + Object compiled = backend.compile(null, cacheKey.idOrCode, + singletonMap(Script.CONTENT_TYPE_OPTION, cacheKey.contentType)); + return new CompiledScript( + scriptType, cacheKey.idOrCode,backend.getType(), compiled); } @Override - protected CompiledScript compileFileScript(String cacheKey, String body, Path file) { + protected CompiledScript compileFileScript(CacheKey cacheKey, String body, Path file) { Object compiled = backend.compile(file.getFileName().toString(), body, emptyMap()); return new CompiledScript(ScriptType.FILE, body, backend.getType(), compiled); @@ -156,11 +172,14 @@ protected CompiledScript compileFileScript(String cacheKey, String body, Path fi * @param type whether to compile ({link ScriptType#INLINE}), lookup from cluster state * ({@link ScriptType#STORED}), or lookup from disk ({@link ScriptType#FILE}) * @param context context in which the template is being run + * @param contentType content type passed to the template backend during compilation * @return the template */ public Function, BytesReference> template(String idOrCode, - ScriptType type, ScriptContext context) { - CompiledScript compiled = compiler.getScript(idOrCode, type, context); + ScriptType type, ScriptContext context, @Nullable String contentType) { + contentType = contentType == null ? DEFAULT_CONTENT_TYPE : contentType; + CacheKey cacheKey = new CacheKey(idOrCode, contentType); + CompiledScript compiled = compiler.getScript(cacheKey, type, context); return params -> { ExecutableScript executable = backend.executable(compiled, params); return (BytesReference) executable.run(); @@ -182,4 +201,38 @@ public void checkCompileBeforeStore(StoredScriptSource source) { public void clusterChanged(ClusterChangedEvent event) { compiler.clusterChanged(event); } + + private static final class CacheKey { + final String idOrCode; + final String contentType; + + private CacheKey(String idOrCode, String contentType) { + this.idOrCode = Objects.requireNonNull(idOrCode); + this.contentType = Objects.requireNonNull(contentType); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + CacheKey cacheKey = (CacheKey)o; + + if (!idOrCode.equals(cacheKey.idOrCode)) return false; + return contentType.equals(cacheKey.contentType); + + } + + @Override + public int hashCode() { + int result = idOrCode.hashCode(); + result = 31 * result + contentType.hashCode(); + return result; + } + + @Override + public String toString() { + return "id=" + idOrCode + ", contentType=" + contentType; + } + } } diff --git a/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java b/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java index 137dc24b336d0..522eb724a8bed 100644 --- a/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestionBuilder.java @@ -630,7 +630,7 @@ public SuggestionContext build(QueryShardContext context) throws IOException { if (this.collateQuery != null) { suggestionContext.setCollateQuery( context.getTemplateService().template(collateQuery.getIdOrCode(), - collateQuery.getType(), ScriptContext.Standard.SEARCH)); + collateQuery.getType(), ScriptContext.Standard.SEARCH, null)); if (this.collateParams != null) { suggestionContext.setCollateScriptParams(this.collateParams); } diff --git a/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java b/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java index 7f12b24a14310..5b51354f27b37 100644 --- a/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java +++ b/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java @@ -102,13 +102,13 @@ public void testFileTemplates() throws IOException { resourceWatcherService.notifyNow(); assertEquals(new BytesArray(body), templateService.template("test", ScriptType.FILE, - ScriptContext.Standard.SEARCH).apply(emptyMap())); + ScriptContext.Standard.SEARCH, null).apply(emptyMap())); Files.delete(testFileWithExt); resourceWatcherService.notifyNow(); Exception e = expectThrows(IllegalArgumentException.class, () -> templateService - .template("test", ScriptType.FILE, ScriptContext.Standard.SEARCH)); + .template("test", ScriptType.FILE, ScriptContext.Standard.SEARCH, null)); assertEquals("unable to find file template [test]", e.getMessage()); } @@ -126,8 +126,9 @@ public void testStoredTemplates() throws IOException { templateService.clusterChanged( new ClusterChangedEvent("test", newState, ClusterState.EMPTY_STATE)); - assertEquals(new BytesArray(body), templateService.template("test", ScriptType.STORED, - ScriptContext.Standard.SEARCH).apply(emptyMap())); + assertEquals(new BytesArray(body), templateService + .template("test", ScriptType.STORED, ScriptContext.Standard.SEARCH, null) + .apply(emptyMap())); ClusterState oldState = newState; newState = ClusterState.builder(oldState) @@ -138,15 +139,16 @@ public void testStoredTemplates() throws IOException { templateService.clusterChanged(new ClusterChangedEvent("test", newState, oldState)); Exception e = expectThrows(ResourceNotFoundException.class, () -> templateService - .template("test", ScriptType.STORED, ScriptContext.Standard.SEARCH)); + .template("test", ScriptType.STORED, ScriptContext.Standard.SEARCH, null)); assertEquals("unable to find template [test] in cluster state", e.getMessage()); } public void testInlineTemplates() throws IOException { String body = "{\"test\":\"test\"}"; - assertEquals(new BytesArray(body), templateService.template(body, ScriptType.INLINE, - ScriptContext.Standard.SEARCH).apply(emptyMap())); + assertEquals(new BytesArray(body), templateService + .template(body, ScriptType.INLINE, ScriptContext.Standard.SEARCH, null) + .apply(emptyMap())); } /** diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheTemplateService.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheTemplateService.java deleted file mode 100644 index 09da37cb8a2b8..0000000000000 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MustacheTemplateService.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.mustache; - -import org.elasticsearch.common.settings.Settings; - -import java.io.Closeable; -import java.io.IOException; - -public class MustacheTemplateService implements Closeable { - private final Settings settings; - private final MustacheScriptEngineService engine; - - public MustacheTemplateService(Settings settings, MustacheScriptEngineService engine) { - this.settings = settings; - this.engine = engine; - } - - @Override - public void close() throws IOException { - engine.close(); - } - -} diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java index e5efd073ec181..a332ca9315bb4 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java @@ -39,9 +39,6 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import java.util.Map; -import java.util.function.Function; - import static java.util.Collections.emptyMap; import static org.elasticsearch.script.ScriptContext.Standard.SEARCH; @@ -66,11 +63,10 @@ public TransportSearchTemplateAction(Settings settings, ThreadPool threadPool, T protected void doExecute(SearchTemplateRequest request, ActionListener listener) { final SearchTemplateResponse response = new SearchTemplateResponse(); try { - Function, BytesReference> template = - templateService.template(request.getScript(), request.getScriptType(), SEARCH); - Map params = - request.getScriptParams() == null ? emptyMap() : request.getScriptParams(); - BytesReference source = template.apply(params); + BytesReference source = templateService + .template(request.getScript(), request.getScriptType(), SEARCH, null) + .apply(request.getScriptParams() == null ? + emptyMap() : request.getScriptParams()); response.setSource(source); if (request.isSimulate()) { diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/ContentTypeTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/ContentTypeTests.java new file mode 100644 index 0000000000000..fdd770f4bf076 --- /dev/null +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/ContentTypeTests.java @@ -0,0 +1,91 @@ +/* + * 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.mustache; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.env.Environment; +import org.elasticsearch.script.ScriptContext; +import org.elasticsearch.script.ScriptContextRegistry; +import org.elasticsearch.script.ScriptEngineRegistry; +import org.elasticsearch.script.ScriptMetrics; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.ScriptSettings; +import org.elasticsearch.script.ScriptType; +import org.elasticsearch.script.TemplateService; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.io.IOException; + +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonMap; + +/** + * Validates that {@code contentType} flows properly from {@link TemplateService} into + * {@link MustacheScriptEngineService} with some basic examples. + */ +public class ContentTypeTests extends ESTestCase { + private TemplateService templateService; + + @Before + public void setupTemplateService() throws IOException { + Settings settings = Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) + .put(ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.getKey(), false) + .put(ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE, 10000) + .build(); + ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(emptyList()); + TemplateService.Backend backend = new MustacheScriptEngineService(); + ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(emptyList()); + ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, backend, + scriptContextRegistry); + templateService = new TemplateService(settings, new Environment(settings), null, backend, + scriptContextRegistry, scriptSettings, new ScriptMetrics()); + } + + public void testNullContentType() { + testCase("test \"test\" test", null); + } + + public void testPlainText() { + testCase("test \"test\" test", CustomMustacheFactory.PLAIN_TEXT_MIME_TYPE); + } + + public void testJson() { + testCase("test \\\"test\\\" test", CustomMustacheFactory.JSON_MIME_TYPE); + } + + public void testJsonWithCharset() { + testCase("test \\\"test\\\" test", CustomMustacheFactory.JSON_MIME_TYPE_WITH_CHARSET); + } + + public void testFormEncoded() { + testCase("test %22test%22 test", CustomMustacheFactory.X_WWW_FORM_URLENCODED_MIME_TYPE); + } + + private void testCase(String expected, String contentType) { + assertEquals(expected, templateService + .template("test {{param}} test", ScriptType.INLINE, + ScriptContext.Standard.SEARCH, contentType) + .apply(singletonMap("param", "\"test\"")).utf8ToString()); + + } + +} diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/CustomMustacheFactoryTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/CustomMustacheFactoryTests.java index 265edb7d53bd4..dec76688b8e58 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/CustomMustacheFactoryTests.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/CustomMustacheFactoryTests.java @@ -20,8 +20,8 @@ package org.elasticsearch.script.mustache; import com.github.mustachejava.Mustache; + import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.script.CompiledScript; import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.Script; @@ -52,7 +52,7 @@ public void testCreateEncoder() { assertThat(e.getMessage(), equalTo("No encoder found for MIME type [test]")); assertThat(CustomMustacheFactory.createEncoder(CustomMustacheFactory.JSON_MIME_TYPE_WITH_CHARSET), - instanceOf(CustomMustacheFactory.JsonEscapeEncoder.class)); + instanceOf(CustomMustacheFactory.JsonEscapeEncoder.class)); assertThat(CustomMustacheFactory.createEncoder(CustomMustacheFactory.JSON_MIME_TYPE), instanceOf(CustomMustacheFactory.JsonEscapeEncoder.class)); assertThat(CustomMustacheFactory.createEncoder(CustomMustacheFactory.PLAIN_TEXT_MIME_TYPE), From 4051fe2453ad0edc036a65094ee67d2e3202dc34 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sat, 25 Mar 2017 11:03:34 -0400 Subject: [PATCH 18/37] Notes --- .../main/java/org/elasticsearch/script/TemplateService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index 28072a2025164..d8ce2261fffa7 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -101,8 +101,8 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, return null; } - /* This process throws away contentType which is maybe bad, but it is the behavior - * that we've had in templates for a long time. */ + /* This process throws away which is fine because you aren't allowed to specify it + * when using a stored template anyway. */ String id = cacheKey.idOrCode; // search template requests can possibly pass in the entire path instead // of just an id for looking up a stored script, so we parse the path and From 071488979aef188ab33080d243d879f1a30cbede Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Sat, 25 Mar 2017 11:04:41 -0400 Subject: [PATCH 19/37] Line length --- .../main/java/org/elasticsearch/script/TemplateService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index d8ce2261fffa7..5b51ca3629be5 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -123,7 +123,8 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, } @Override - protected boolean areAnyScriptContextsEnabled(CacheKey cacheKey, ScriptType scriptType) { + protected boolean areAnyScriptContextsEnabled( + CacheKey cacheKey, ScriptType scriptType) { for (ScriptContext scriptContext : scriptContextRegistry.scriptContexts()) { if (scriptPermits.checkContextPermissions(backend.getType(), scriptType, scriptContext)) { From 8d1165e1f4242362b3ebe6123bc5b7d74c8e60f7 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 14:33:59 -0400 Subject: [PATCH 20/37] Remove logging guards They don't help in log4j2. They are a leftover from a bygone age. --- .../elasticsearch/script/CachingCompiler.java | 18 +++++------------- .../script/TemplateServiceTests.java | 6 ++++-- 2 files changed, 9 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 7d44ecdd171ff..5e2385f92f71f 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -102,9 +102,7 @@ public CachingCompiler(Settings settings, Environment env, // add file watcher for file scripts and templates scriptsDirectory = env.scriptsFile(); - if (logger.isTraceEnabled()) { - logger.trace("Using scripts directory [{}] ", scriptsDirectory); - } + logger.trace("Using scripts directory [{}] ", scriptsDirectory); FileWatcher fileWatcher = new FileWatcher(scriptsDirectory); fileWatcher.addListener(new DirectoryChangesListener()); if (ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING.get(settings)) { @@ -213,9 +211,7 @@ public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, } try { - if (logger.isTraceEnabled()) { - logger.trace("compiling [{}]", cacheKey); - } + logger.trace("compiling [{}]", cacheKey); // Check whether too many compilations have happened checkCompilationLimit(); compiledScript = compile(scriptType, cacheKey); @@ -274,10 +270,8 @@ public final void clusterChanged(ClusterChangedEvent event) { private class CacheRemovalListener implements RemovalListener { @Override public void onRemoval(RemovalNotification notification) { - if (logger.isDebugEnabled()) { - logger.debug("removed {} from cache, reason: {}", notification.getValue(), - notification.getRemovalReason()); - } + logger.debug("removed {} from cache, reason: {}", notification.getValue(), + notification.getRemovalReason()); scriptMetrics.onCacheEviction(); } } @@ -307,9 +301,7 @@ public void onFileInit(Path file) { logger.debug("Skipped script with invalid extension : [{}]", file); return; } - if (logger.isTraceEnabled()) { - logger.trace("Loading script file : [{}]", file); - } + logger.trace("Loading script file : [{}]", file); CacheKeyT cacheKey = cacheKeyForFile(scriptNameExt.v1(), scriptNameExt.v2()); if (cacheKey == null) { diff --git a/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java b/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java index 5b51354f27b37..05311fae427be 100644 --- a/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java +++ b/core/src/test/java/org/elasticsearch/script/TemplateServiceTests.java @@ -109,7 +109,8 @@ public void testFileTemplates() throws IOException { Exception e = expectThrows(IllegalArgumentException.class, () -> templateService .template("test", ScriptType.FILE, ScriptContext.Standard.SEARCH, null)); - assertEquals("unable to find file template [test]", e.getMessage()); + assertEquals("unable to find file template [id=test, contentType=text/plain]", + e.getMessage()); } public void testStoredTemplates() throws IOException { @@ -140,7 +141,8 @@ public void testStoredTemplates() throws IOException { Exception e = expectThrows(ResourceNotFoundException.class, () -> templateService .template("test", ScriptType.STORED, ScriptContext.Standard.SEARCH, null)); - assertEquals("unable to find template [test] in cluster state", e.getMessage()); + assertEquals("unable to find template [id=test, contentType=text/plain] in cluster state", + e.getMessage()); } public void testInlineTemplates() throws IOException { From f0d70bc1a86f5d16a794e34709019e78de41bcc9 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 14:40:40 -0400 Subject: [PATCH 21/37] Remove leftover. --- .../org/elasticsearch/common/xcontent/XContentFactory.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java b/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java index 0e6a13e60e022..dc9d1c493a3ed 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/XContentFactory.java @@ -277,6 +277,9 @@ public static XContentType xContentType(byte[] data, int offset, int length) { @Deprecated public static XContent xContent(BytesReference bytes) { XContentType type = xContentType(bytes); + if (type == null) { + throw new ElasticsearchParseException("Failed to derive xcontent"); + } return xContent(type); } From eccf17db8c663c21b8315fd5e792fc802e07cd61 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 14:42:13 -0400 Subject: [PATCH 22/37] Rename method --- .../main/java/org/elasticsearch/script/CachingCompiler.java | 6 +++--- .../main/java/org/elasticsearch/script/ScriptService.java | 2 +- .../main/java/org/elasticsearch/script/TemplateService.java | 3 +-- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 5e2385f92f71f..96b5d697f2c13 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -137,7 +137,7 @@ protected abstract StoredScriptSource lookupStoredScript(ClusterState clusterSta * to reject compilation if all script contexts are disabled and produce a nice error message * earlier rather than later. */ - protected abstract boolean areAnyScriptContextsEnabled(CacheKeyT cacheKey, + protected abstract boolean anyScriptContextsEnabled(CacheKeyT cacheKey, ScriptType scriptType); /** @@ -243,7 +243,7 @@ private CacheKeyT getScriptFromClusterState(CacheKeyT cacheKey) { public final void checkCompileBeforeStore(StoredScriptSource source) { CacheKeyT cacheKey = cacheKeyFromClusterState(source); try { - if (areAnyScriptContextsEnabled(cacheKey, ScriptType.STORED)) { + if (anyScriptContextsEnabled(cacheKey, ScriptType.STORED)) { Object compiled = compile(ScriptType.STORED, cacheKey); if (compiled == null) { @@ -311,7 +311,7 @@ public void onFileInit(Path file) { /* we don't know yet what the script will be used for, but if all of the operations * for this lang with file scripts are disabled, it makes no sense to even compile * it and cache it. */ - if (areAnyScriptContextsEnabled(cacheKey, ScriptType.FILE)) { + if (anyScriptContextsEnabled(cacheKey, ScriptType.FILE)) { logger.info("compiling script file [{}]", file.toAbsolutePath()); try (InputStreamReader reader = new InputStreamReader( Files.newInputStream(file), StandardCharsets.UTF_8)) { diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index 53e54ffd97a7d..2dfc656f48ce6 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -167,7 +167,7 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, Cache } @Override - protected boolean areAnyScriptContextsEnabled(CacheKey cacheKey, ScriptType scriptType) { + protected boolean anyScriptContextsEnabled(CacheKey cacheKey, ScriptType scriptType) { for (ScriptContext scriptContext : scriptContextRegistry.scriptContexts()) { if (scriptPermits.checkContextPermissions(cacheKey.lang, scriptType, scriptContext)) { return true; diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index 5b51ca3629be5..224cab6fa377f 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -123,8 +123,7 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, } @Override - protected boolean areAnyScriptContextsEnabled( - CacheKey cacheKey, ScriptType scriptType) { + protected boolean anyScriptContextsEnabled(CacheKey cacheKey, ScriptType scriptType) { for (ScriptContext scriptContext : scriptContextRegistry.scriptContexts()) { if (scriptPermits.checkContextPermissions(backend.getType(), scriptType, scriptContext)) { From 6f00573fe3383601c39828dc5333bcc8f611694d Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 14:45:03 -0400 Subject: [PATCH 23/37] requireNonNull with messages --- .../main/java/org/elasticsearch/script/CachingCompiler.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 96b5d697f2c13..9f6b6fbe935e5 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -175,7 +175,9 @@ protected abstract void checkContextPermissions(CacheKeyT cacheKey, ScriptType s public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, ScriptContext scriptContext) { - Objects.requireNonNull(cacheKey); + Objects.requireNonNull(cacheKey, "cacheKey is required"); + Objects.requireNonNull(scriptType, "scriptType is required"); + Objects.requireNonNull(scriptContext, "scriptContext is required"); // First resolve stored scripts so so we have accurate parameters for checkCanExecuteScript if (scriptType == ScriptType.STORED) { From ac1b823489541ea492647a05ab8dff97b33db9ad Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 14:51:03 -0400 Subject: [PATCH 24/37] Better comments in `checkCompileBeforeStore` Also drop an unnecessary null check. --- .../elasticsearch/script/CachingCompiler.java | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 9f6b6fbe935e5..43796fb31812f 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -246,18 +246,20 @@ public final void checkCompileBeforeStore(StoredScriptSource source) { CacheKeyT cacheKey = cacheKeyFromClusterState(source); try { if (anyScriptContextsEnabled(cacheKey, ScriptType.STORED)) { - Object compiled = compile(ScriptType.STORED, cacheKey); - - if (compiled == null) { - throw new IllegalArgumentException("failed to parse/compile"); - } + /* Compile the script to make sure it compiles but through away the result. We'll + * populate the cache if it is ever used. */ + compile(ScriptType.STORED, cacheKey); } else { - throw new IllegalArgumentException("cannot be run under any context"); + throw new IllegalArgumentException(type + " cannot be run under any context"); } - } catch (ScriptException good) { - throw good; + } catch (ScriptException betterException) { + throw betterException; } catch (Exception exception) { - throw new IllegalArgumentException("failed to parse/compile", exception); + /* Catch any less fancy scripting exceptions and send them back to the user with a + * little context information. We'd *love* to only ever through ScriptException here but + * that requires information not available to all extensions of this class. So we make + * do here. */ + throw new IllegalArgumentException("failed to compile " + type, exception); } } From 9360cfa0688f8e1c391f03f3be05b126424cc1c7 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 14:54:15 -0400 Subject: [PATCH 25/37] No extension --- .../main/java/org/elasticsearch/script/CachingCompiler.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index 43796fb31812f..b000d74dadb54 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -27,6 +27,7 @@ import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.cache.Cache; import org.elasticsearch.common.cache.CacheBuilder; import org.elasticsearch.common.cache.RemovalListener; @@ -289,7 +290,9 @@ private Tuple getScriptNameExt(Path file) { } String ext = scriptPath.toString().substring(extIndex + 1); - if (ext.isEmpty()) { + if (Strings.hasText(ext) == false) { + /* Files without extensions or with degenerate extensions like " " are not scripts + * or templates so we silently skip them rather than try to compile them. */ return null; } From 558b26b3f5748ca487b2b89cf92fbdaf51718790 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 14:55:20 -0400 Subject: [PATCH 26/37] Change exception type --- .../src/main/java/org/elasticsearch/script/ScriptModule.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/ScriptModule.java b/core/src/main/java/org/elasticsearch/script/ScriptModule.java index 70faab47afff9..951a640e4468d 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptModule.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptModule.java @@ -19,6 +19,7 @@ package org.elasticsearch.script; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; @@ -91,7 +92,7 @@ public ScriptModule(Settings settings, Environment environment, scriptService = new ScriptService(settings, environment, resourceWatcherService, scriptEngineRegistry, scriptContextRegistry, scriptSettings, scriptMetrics); } catch (IOException e) { - throw new RuntimeException("Couldn't setup ScriptService", e); + throw new ElasticsearchException("Couldn't setup ScriptService", e); } if (templateBackend == null) { @@ -101,7 +102,7 @@ public ScriptModule(Settings settings, Environment environment, templateService = new TemplateService(settings, environment, resourceWatcherService, templateBackend, scriptContextRegistry, scriptSettings, scriptMetrics); } catch (IOException e) { - throw new RuntimeException("Couldn't setup TemplateService", e); + throw new ElasticsearchException("Couldn't setup TemplateService", e); } } From 8a58c24ca43e4620b77c0da5ff5f2081bbca40da Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 15:00:09 -0400 Subject: [PATCH 27/37] Some javadoc --- .../main/java/org/elasticsearch/script/ScriptModule.java | 6 ++++++ .../main/java/org/elasticsearch/script/TemplateService.java | 5 +++++ 2 files changed, 11 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/script/ScriptModule.java b/core/src/main/java/org/elasticsearch/script/ScriptModule.java index 951a640e4468d..a1fa53673e4b2 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptModule.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptModule.java @@ -134,6 +134,12 @@ public void registerClusterSettingsListeners(ClusterSettings clusterSettings) { scriptService.registerClusterSettingsListeners(clusterSettings); } + /** + * Template backend that refuses to compile templates. Useful for starting Elasticsearch without + * the {@code lang-mustache} module. Which we do frequently during testing. Technically it'd be + * used if you started Elasticsearch without {@code lang-mustache} installed but that isn't a + * supported deployment. + */ private static class TemplatesUnsupportedBackend implements TemplateService.Backend { @Override public String getType() { diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index 224cab6fa377f..e86425f9b5d2d 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -45,6 +45,11 @@ * Source of templates (think Mustache or StringTemplate). */ public class TemplateService implements ClusterStateListener { + /** + * Backend that implements templates. Must be provided by a module. The only tested + * implementation of this is the one provided by {@code lang-mustache} and the noop + * implementation in {@link ScriptModule}. + */ public interface Backend extends ScriptEngineService {} // TODO customize this for templates private static final String DEFAULT_CONTENT_TYPE = "text/plain"; From d7ff55bad2248a2fa0b41b42df0d71654b46c2e2 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 15:08:29 -0400 Subject: [PATCH 28/37] More javadocs --- .../elasticsearch/script/ScriptService.java | 25 ++++++++++++++++--- .../elasticsearch/script/TemplateService.java | 15 +++++++++++ 2 files changed, 37 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index 2dfc656f48ce6..bfc54f36651e3 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -54,6 +54,9 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; +/** + * Source of scripts that are run in various parts of Elasticsearch. + */ public class ScriptService extends AbstractComponent implements Closeable, ClusterStateListener { static final String DISABLE_DYNAMIC_SCRIPTING_SETTING = "script.disable_dynamic"; @@ -78,9 +81,25 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust private final CachingCompiler compiler; private final int maxScriptSizeInBytes; - public ScriptService(Settings settings, Environment env, ResourceWatcherService resourceWatcherService, - ScriptEngineRegistry scriptEngineRegistry, ScriptContextRegistry scriptContextRegistry, ScriptSettings scriptSettings, - ScriptMetrics scriptMetrics) throws IOException { + /** + * Build the service. + * + * @param settings common settings loaded at node startup + * @param env environment in which the node is running. Used to resolve the + * {@code config/scripts} directory that is scanned periodically for scripts. + * @param resourceWatcherService Scans the {@code config/scripts} directory. + * @param scriptEngineRegistry all {@link ScriptEngineService}s that we support. This delegates + * to those engines to build the actual executable. + * @param scriptContextRegistry all {@link ScriptContext}s that we support. + * @param scriptSettings settings for scripts + * @param scriptMetrics compilation metrics for scripts. This should be shared between + * {@link ScriptService} and {@link TemplateService} + * @throws IOException If there is an error scanning the {@code config/scripts} directory. + */ + public ScriptService(Settings settings, Environment env, + ResourceWatcherService resourceWatcherService, + ScriptEngineRegistry scriptEngineRegistry, ScriptContextRegistry scriptContextRegistry, + ScriptSettings scriptSettings, ScriptMetrics scriptMetrics) throws IOException { super(settings); if (Strings.hasLength(settings.get(DISABLE_DYNAMIC_SCRIPTING_SETTING))) { throw new IllegalArgumentException(DISABLE_DYNAMIC_SCRIPTING_SETTING + " is not a supported setting, replace with " diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index e86425f9b5d2d..430114365f21e 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -59,6 +59,21 @@ public interface Backend extends ScriptEngineService {} // TODO customize this f private final ScriptPermits scriptPermits; private final CachingCompiler compiler; + /** + * Build the service. + * + * @param settings common settings loaded at node startup + * @param env environment in which the node is running. Used to resolve the + * {@code config/scripts} directory that is scanned periodically for scripts. + * @param resourceWatcherService Scans the {@code config/scripts} directory. + * @param backend the backend that actually compiles the templates. + * @param scriptEngineRegistry all {@link ScriptEngineService}s that we support. This delegates + * to those engines to build the actual executable. + * @param scriptSettings settings for scripts + * @param scriptMetrics compilation metrics for scripts. This should be shared between + * {@link ScriptService} and {@link TemplateService} + * @throws IOException If there is an error scanning the {@code config/scripts} directory. + */ public TemplateService(Settings settings, Environment env, ResourceWatcherService resourceWatcherService, Backend backend, ScriptContextRegistry scriptContextRegistry, ScriptSettings scriptSettings, From f74d9d6e7acd3aa7c82647454aab44ec5b836748 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 15:13:10 -0400 Subject: [PATCH 29/37] Drop funny parsing for scripts that should only be there for templates --- .../org/elasticsearch/script/ScriptService.java | 17 ++--------------- .../elasticsearch/script/TemplateService.java | 2 -- 2 files changed, 2 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index bfc54f36651e3..b0469776ed9b9 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -133,7 +133,7 @@ protected CacheKey cacheKeyForFile(String baseName, String extension) { } ScriptEngineService engine = scriptEnginesByExt.get(extension); if (engine == null) { - logger.warn("script file extension not supported [" + extension + "]"); + logger.warn("script file extension not supported [{}.{}]", baseName, extension); return null; } return new CacheKey(engine.getType(), baseName, null); @@ -163,22 +163,9 @@ protected StoredScriptSource lookupStoredScript(ClusterState clusterState, Cache } String id = cacheKey.idOrCode; - // search template requests can possibly pass in the entire path instead - // of just an id for looking up a stored script, so we parse the path and - // check for appropriate errors String[] path = id.split("/"); - if (path.length == 3) { - if (cacheKey.lang != null && cacheKey.lang.equals(path[1]) == false) { - throw new IllegalStateException("conflicting script languages, found [" + path[1] - + "] but expected [" + cacheKey.lang + "]"); - } - - id = path[2]; - - deprecationLogger.deprecated("use of [" + cacheKey.idOrCode + "] for looking up" + - " stored scripts/templates has been deprecated, use only [" + id + "] instead"); - } else if (path.length != 1) { + if (path.length != 1) { throw new IllegalArgumentException("illegal stored script format [" + id + "] use only "); } diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index 430114365f21e..2b23411bba7c2 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -67,8 +67,6 @@ public interface Backend extends ScriptEngineService {} // TODO customize this f * {@code config/scripts} directory that is scanned periodically for scripts. * @param resourceWatcherService Scans the {@code config/scripts} directory. * @param backend the backend that actually compiles the templates. - * @param scriptEngineRegistry all {@link ScriptEngineService}s that we support. This delegates - * to those engines to build the actual executable. * @param scriptSettings settings for scripts * @param scriptMetrics compilation metrics for scripts. This should be shared between * {@link ScriptService} and {@link TemplateService} From cdecc358acf668228fe4b3c5369337022a1e39f1 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 15:19:20 -0400 Subject: [PATCH 30/37] Move null checks for stored templates or scripts Move them to the superclass and move them to Optional --- .../elasticsearch/script/CachingCompiler.java | 18 ++++++++++-------- .../elasticsearch/script/ScriptService.java | 14 +------------- .../elasticsearch/script/TemplateService.java | 16 +--------------- 3 files changed, 12 insertions(+), 36 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java index b000d74dadb54..9843b1f9d0ed4 100644 --- a/core/src/main/java/org/elasticsearch/script/CachingCompiler.java +++ b/core/src/main/java/org/elasticsearch/script/CachingCompiler.java @@ -52,6 +52,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.ConcurrentMap; /** @@ -130,7 +131,7 @@ public CachingCompiler(Settings settings, Environment env, * Lookup a stored script (or template) from the cluster state, returning null if it is not * found. */ - protected abstract StoredScriptSource lookupStoredScript(ClusterState clusterState, + protected abstract StoredScriptSource lookupStoredScript(ScriptMetaData scriptMetaData, CacheKeyT cacheKey); /** @@ -231,13 +232,14 @@ public final CompiledScript getScript(CacheKeyT cacheKey, ScriptType scriptType, } private CacheKeyT getScriptFromClusterState(CacheKeyT cacheKey) { - StoredScriptSource source = lookupStoredScript(clusterState, cacheKey); - - if (source == null) { - throw new ResourceNotFoundException( - "unable to find " + type + " [" + cacheKey + "] in cluster state"); - } - return cacheKeyFromClusterState(source); + StoredScriptSource resolved = Optional.ofNullable(clusterState) + .map(ClusterState::metaData) + .map(metaData -> (ScriptMetaData) metaData.custom(ScriptMetaData.TYPE)) + .map(scriptMetaData -> lookupStoredScript(scriptMetaData, cacheKey)) + .orElseThrow(() -> new ResourceNotFoundException( + "unable to find " + type + " [" + cacheKey + "] in cluster state")); + + return cacheKeyFromClusterState(resolved); } /** diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index b0469776ed9b9..0a49694ca9b0a 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -145,19 +145,7 @@ protected CacheKey cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { } @Override - protected StoredScriptSource lookupStoredScript(ClusterState clusterState, CacheKey cacheKey) { - if (clusterState == null) { - return null; - } - MetaData metaData = clusterState.metaData(); - if (metaData == null) { - return null; - } - ScriptMetaData scriptMetaData = clusterState.metaData().custom(ScriptMetaData.TYPE); - if (scriptMetaData == null) { - return null; - } - + protected StoredScriptSource lookupStoredScript(ScriptMetaData scriptMetaData, CacheKey cacheKey) { if (cacheKey.lang != null && isLangSupported(cacheKey.lang) == false) { throw new IllegalArgumentException("unable to get stored script with unsupported lang [" + cacheKey.lang + "]"); } diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index 2b23411bba7c2..6076eef6818ff 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -21,9 +21,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateListener; -import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.logging.DeprecationLogger; @@ -105,20 +103,8 @@ protected CacheKey cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { } @Override - protected StoredScriptSource lookupStoredScript(ClusterState clusterState, + protected StoredScriptSource lookupStoredScript(ScriptMetaData scriptMetaData, CacheKey cacheKey) { - if (clusterState == null) { - return null; - } - MetaData metaData = clusterState.metaData(); - if (metaData == null) { - return null; - } - ScriptMetaData scriptMetaData = clusterState.metaData().custom(ScriptMetaData.TYPE); - if (scriptMetaData == null) { - return null; - } - /* This process throws away which is fine because you aren't allowed to specify it * when using a stored template anyway. */ String id = cacheKey.idOrCode; From 83aaccec7abb37e2b82ee21cc883f21095c17612 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 15:21:10 -0400 Subject: [PATCH 31/37] Code reuse --- .../main/java/org/elasticsearch/script/ScriptService.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index 0a49694ca9b0a..2cbdca5e6646b 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -206,10 +206,7 @@ protected CompiledScript compileFileScript(CacheKey cacheKey, String body, Path } private CompiledScript compile(ScriptType scriptType, CacheKey cacheKey, String body, String fileName) { - ScriptEngineService engine = scriptEnginesByLang.get(cacheKey.lang); - if (engine == null) { - throw new IllegalArgumentException("script lang not supported [" + cacheKey.lang + "]"); - } + ScriptEngineService engine = getScriptEngineServiceForLang(cacheKey.lang); Object executable = engine.compile(fileName, body, cacheKey.options); return new CompiledScript(scriptType, body, engine.getType(), executable); } From 1d16334610a9ca884d5820babbd0e3350fc46820 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 15:27:52 -0400 Subject: [PATCH 32/37] Javadocs --- .../elasticsearch/script/ScriptService.java | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index 2cbdca5e6646b..c7416c248e013 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -32,6 +32,7 @@ import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.settings.ClusterSettings; @@ -47,6 +48,7 @@ import java.io.IOException; import java.nio.file.Path; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -286,6 +288,10 @@ public void clusterChanged(ClusterChangedEvent event) { compiler.clusterChanged(event); } + /** + * Fetch a stored script from a cluster state. + */ + @Nullable public final StoredScriptSource getStoredScript(ClusterState state, GetStoredScriptRequest request) { ScriptMetaData scriptMetadata = state.metaData().custom(ScriptMetaData.TYPE); @@ -296,6 +302,9 @@ public final StoredScriptSource getStoredScript(ClusterState state, GetStoredScr } } + /** + * Put a stored script in the cluster state. + */ public void putStoredScript(ClusterService clusterService, PutStoredScriptRequest request, ActionListener listener) { if (request.content().length() > maxScriptSizeInBytes) { @@ -329,6 +338,9 @@ public ClusterState execute(ClusterState currentState) throws Exception { }); } + /** + * Delete a stored script from the cluster state. + */ public void deleteStoredScript(ClusterService clusterService, DeleteStoredScriptRequest request, ActionListener listener) { if (request.lang() != null && isLangSupported(request.lang()) == false) { @@ -358,7 +370,19 @@ private static final class CacheKey { final String idOrCode; final Map options; - private CacheKey(String lang, String idOrCode, Map options) { + /** + * Build the cache key. + * + * @param lang language of the script + * @param idOrCode id of the script or code for the script. For file based scripts this is + * always the id, for inline scripts this is always the code. For stored scripts this + * should be built as the id and then use + * {@link CachingCompiler#lookupStoredScript(ScriptMetaData, Object)} to replace the + * id with the source. + * @param options map of options used during script compilation. {@code null} is translated + * to {@link Collections#emptyMap()} on construction for ease of use. + */ + private CacheKey(String lang, String idOrCode, @Nullable Map options) { this.lang = lang; this.idOrCode = idOrCode; this.options = options == null ? emptyMap() : options; From 2f1c935d502711c08fced5201a33b0b8ca1eab61 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 15:29:24 -0400 Subject: [PATCH 33/37] StringBuilder --- .../main/java/org/elasticsearch/script/ScriptService.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index c7416c248e013..4e498002d7d43 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -411,11 +411,12 @@ public int hashCode() { @Override public String toString() { - String result = "lang=" + lang + ", id=" + idOrCode; + StringBuilder result = new StringBuilder("lang=").append(lang); + result.append(", id=").append(idOrCode); if (false == options.isEmpty()) { - result += ", options " + options; + result.append(", options ").append(options); } - return result; + return result.toString(); } } } From b6f9cd575e5d52204c1e39444fb5a240536b4be8 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 15:34:38 -0400 Subject: [PATCH 34/37] Messages --- .../main/java/org/elasticsearch/script/ScriptService.java | 4 ++-- .../java/org/elasticsearch/script/ScriptSettings.java | 1 + .../java/org/elasticsearch/script/TemplateService.java | 8 +++----- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index 4e498002d7d43..67672e615e2f9 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -109,9 +109,9 @@ public ScriptService(Settings settings, Environment env, + "`script.disable_dynamic: false` with `script.inline: true` and `script.stored: true` in elasticsearch.yml"); } - Objects.requireNonNull(scriptEngineRegistry); + Objects.requireNonNull(scriptEngineRegistry, "scriptEngineRegistry is required"); this.scriptEngines = scriptEngineRegistry.getRegisteredLanguages().values(); - Objects.requireNonNull(scriptContextRegistry); + Objects.requireNonNull(scriptContextRegistry, "scriptContextRegistry is required"); Map enginesByLangBuilder = new HashMap<>(); Map enginesByExtBuilder = new HashMap<>(); diff --git a/core/src/main/java/org/elasticsearch/script/ScriptSettings.java b/core/src/main/java/org/elasticsearch/script/ScriptSettings.java index f1c5ac9aa8b44..cf0d1b26523e1 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptSettings.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptSettings.java @@ -142,6 +142,7 @@ private static List> languageSettings(Map c : scriptEngineRegistry.getRegisteredScriptEngineServices()) { if (c != NativeScriptEngineService.class) { + // native scripts are always enabled, and their settings can not be changed String language = scriptEngineRegistry.getLanguage(c); populate.accept(language, scriptEngineRegistry.getDefaultInlineScriptEnableds().get(language)); } diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index 6076eef6818ff..a159a290cb6ed 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -74,7 +74,7 @@ public TemplateService(Settings settings, Environment env, ResourceWatcherService resourceWatcherService, Backend backend, ScriptContextRegistry scriptContextRegistry, ScriptSettings scriptSettings, ScriptMetrics scriptMetrics) throws IOException { - Objects.requireNonNull(scriptContextRegistry); + Objects.requireNonNull(scriptContextRegistry, "scriptContextRegistry is required"); this.backend = backend; this.scriptPermits = new ScriptPermits(settings, scriptSettings, scriptContextRegistry); @@ -94,10 +94,8 @@ protected CacheKey cacheKeyForFile(String baseName, String extension) { protected CacheKey cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { String contentType = DEFAULT_CONTENT_TYPE; if (scriptMetadata.getOptions() != null) { - contentType = scriptMetadata.getOptions().get(Script.CONTENT_TYPE_OPTION); - if (contentType == null) { - contentType = DEFAULT_CONTENT_TYPE; - } + contentType = scriptMetadata.getOptions() + .getOrDefault(Script.CONTENT_TYPE_OPTION, DEFAULT_CONTENT_TYPE); } return new CacheKey(scriptMetadata.getCode(), contentType); } From 83183937d3292c888cc94a6b90d902fa7670a12f Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 15:36:03 -0400 Subject: [PATCH 35/37] Fix comment --- .../main/java/org/elasticsearch/script/TemplateService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index a159a290cb6ed..262c2a5387576 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -103,8 +103,8 @@ protected CacheKey cacheKeyFromClusterState(StoredScriptSource scriptMetadata) { @Override protected StoredScriptSource lookupStoredScript(ScriptMetaData scriptMetaData, CacheKey cacheKey) { - /* This process throws away which is fine because you aren't allowed to specify it - * when using a stored template anyway. */ + /* This process throws away cacheKey.contentType which is fine because you aren't + * allowed to specify it when using a stored template anyway. */ String id = cacheKey.idOrCode; // search template requests can possibly pass in the entire path instead // of just an id for looking up a stored script, so we parse the path and From 7ab675df99981ea1729957c453b51a6c4306d379 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 15:44:39 -0400 Subject: [PATCH 36/37] Fix warning message --- .../main/java/org/elasticsearch/script/TemplateService.java | 4 ++-- .../org/elasticsearch/script/mustache/SearchTemplateIT.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index 262c2a5387576..b48862b26c257 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -114,8 +114,8 @@ protected StoredScriptSource lookupStoredScript(ScriptMetaData scriptMetaData, if (path.length == 3) { id = path[2]; deprecationLogger.deprecated("use of [{}] for looking up stored " - + "scripts/templates has been deprecated, use only [{}] instead", - cacheKey, id); + + "templates has been deprecated, use only [{}] instead", + cacheKey.idOrCode, id); } else if (path.length != 1) { throw new IllegalArgumentException( "illegal stored script format [" + id + "] use only "); diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java index b5411a5e768e2..ce84b822ddebb 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/SearchTemplateIT.java @@ -285,7 +285,7 @@ public void testIndexedTemplate() throws Exception { .get(); assertHitCount(searchResponse.getResponse(), 1); assertWarnings("use of [/mustache/2] for looking up" + - " stored scripts/templates has been deprecated, use only [2] instead"); + " stored templates has been deprecated, use only [2] instead"); Map vars = new HashMap<>(); vars.put("fieldParam", "bar"); From c9bb0b098752a09b1f647c90588ca9bd25880992 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 27 Mar 2017 16:09:14 -0400 Subject: [PATCH 37/37] Fix broken tests --- .../src/main/java/org/elasticsearch/script/TemplateService.java | 2 +- .../test/lang_mustache/20_render_search_template.yaml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/script/TemplateService.java b/core/src/main/java/org/elasticsearch/script/TemplateService.java index b48862b26c257..47bfbb7f55f0b 100644 --- a/core/src/main/java/org/elasticsearch/script/TemplateService.java +++ b/core/src/main/java/org/elasticsearch/script/TemplateService.java @@ -50,7 +50,7 @@ public class TemplateService implements ClusterStateListener { */ public interface Backend extends ScriptEngineService {} // TODO customize this for templates - private static final String DEFAULT_CONTENT_TYPE = "text/plain"; + private static final String DEFAULT_CONTENT_TYPE = "application/json"; private static final Logger logger = ESLoggerFactory.getLogger(TemplateService.class); private static final DeprecationLogger deprecationLogger = new DeprecationLogger(logger); private final Backend backend; diff --git a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/20_render_search_template.yaml b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/20_render_search_template.yaml index 8243c1a568873..6d26b2ca2d01c 100644 --- a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/20_render_search_template.yaml +++ b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/20_render_search_template.yaml @@ -152,6 +152,6 @@ --- "Missing file template": - do: - catch: /unable.to.find.file.template.\[missing\]/ + catch: /unable.to.find.file.template.\[id=missing, contentType=text/plain\]/ search_template: body: { "file" : "missing"}