diff --git a/docs/painless/painless-execute-script.asciidoc b/docs/painless/painless-execute-script.asciidoc index a3ac5b578d781..2aca959778699 100644 --- a/docs/painless/painless-execute-script.asciidoc +++ b/docs/painless/painless-execute-script.asciidoc @@ -9,23 +9,24 @@ The Painless execute API allows an arbitrary script to be executed and a result .Parameters [options="header"] |====== -| Name | Required | Default | Description -| `script` | yes | - | The script to execute -| `context` | no | `painless_test` | The context the script should be executed in. +| Name | Required | Default | Description +| `script` | yes | - | The script to execute +| `context` | no | `painless_test` | The context the script should be executed in. +| `context_setup` | no | - | Additional parameters to the context. |====== ==== Contexts Contexts control how scripts are executed, what variables are available at runtime and what the return type is. -===== Painless test script context +===== Painless test context The `painless_test` context executes scripts as is and do not add any special parameters. The only variable that is available is `params`, which can be used to access user defined values. The result of the script is always converted to a string. If no context is specified then this context is used by default. -==== Example +====== Example Request: @@ -52,4 +53,124 @@ Response: "result": "0.1" } -------------------------------------------------- -// TESTRESPONSE \ No newline at end of file +// TESTRESPONSE + +===== Filter context + +The `filter` context executes scripts as if they were executed inside a `script` query. +For testing purposes a document must be provided that will be indexed temporarily in-memory and +is accessible to the script being tested. Because of this the _source, stored fields and doc values +are available in the script being tested. + +The following parameters may be specified in `context_setup` for a filter context: + +document:: Contains the document that will be temporarily indexed in-memory and is accessible from the script. +index:: The name of an index containing a mapping that is compatable with the document being indexed. + +====== Example + +[source,js] +---------------------------------------------------------------- +PUT /my-index +{ + "mappings": { + "_doc": { + "properties": { + "field": { + "type": "keyword" + } + } + } + } +} + +POST /_scripts/painless/_execute +{ + "script": { + "source": "doc['field'].value.length() <= params.max_length", + "params": { + "max_length": 4 + } + }, + "context": "filter", + "context_setup": { + "index": "my-index", + "document": { + "field": "four" + } + } +} +---------------------------------------------------------------- +// CONSOLE + +Response: + +[source,js] +-------------------------------------------------- +{ + "result": true +} +-------------------------------------------------- +// TESTRESPONSE + + +===== Score context + +The `score` context executes scripts as if they were executed inside a `script_score` function in +`function_score` query. + +The following parameters may be specified in `context_setup` for a score context: + +document:: Contains the document that will be temporarily indexed in-memory and is accessible from the script. +index:: The name of an index containing a mapping that is compatable with the document being indexed. +query:: If `_score` is used in the script then a query can specified that will be used to compute a score. + +====== Example + +[source,js] +---------------------------------------------------------------- +PUT /my-index +{ + "mappings": { + "_doc": { + "properties": { + "field": { + "type": "keyword" + }, + "rank": { + "type": "long" + } + } + } + } +} + + +POST /_scripts/painless/_execute +{ + "script": { + "source": "doc['rank'].value / params.max_rank", + "params": { + "max_rank": 5.0 + } + }, + "context": "score", + "context_setup": { + "index": "my-index", + "document": { + "rank": 4 + } + } +} +---------------------------------------------------------------- +// CONSOLE + +Response: + +[source,js] +-------------------------------------------------- +{ + "result": 0.8 +} +-------------------------------------------------- +// TESTRESPONSE diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/PainlessExecuteAction.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/PainlessExecuteAction.java index 01139f6cf2e70..229c919a2e65d 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/PainlessExecuteAction.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/PainlessExecuteAction.java @@ -18,41 +18,75 @@ */ package org.elasticsearch.painless; +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.apache.lucene.store.RAMDirectory; +import org.elasticsearch.Version; import org.elasticsearch.action.Action; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.HandledTransportAction; -import org.elasticsearch.client.ElasticsearchClient; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.single.shard.SingleShardRequest; +import org.elasticsearch.action.support.single.shard.TransportSingleShardAction; import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.routing.ShardsIterator; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.CheckedBiFunction; import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.SourceToParse; +import org.elasticsearch.index.query.AbstractQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesService; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.BytesRestResponse; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestResponse; import org.elasticsearch.rest.action.RestBuilderListener; +import org.elasticsearch.script.FilterScript; +import org.elasticsearch.script.ScoreScript; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptType; -import org.elasticsearch.tasks.Task; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.Locale; +import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -75,40 +109,181 @@ public Response newResponse() { return new Response(); } - public static class Request extends ActionRequest implements ToXContent { + public static class Request extends SingleShardRequest implements ToXContent { private static final ParseField SCRIPT_FIELD = new ParseField("script"); private static final ParseField CONTEXT_FIELD = new ParseField("context"); + private static final ParseField CONTEXT_SETUP_FIELD = new ParseField("context_setup"); private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - "painless_execute_request", args -> new Request((Script) args[0], (SupportedContext) args[1])); + "painless_execute_request", args -> new Request((Script) args[0], (String) args[1], (ContextSetup) args[2])); static { PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> Script.parse(p), SCRIPT_FIELD); - PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), (p, c) -> { - // For now only accept an empty json object: - XContentParser.Token token = p.nextToken(); - assert token == XContentParser.Token.FIELD_NAME; - String contextType = p.currentName(); - token = p.nextToken(); - assert token == XContentParser.Token.START_OBJECT; - token = p.nextToken(); - assert token == XContentParser.Token.END_OBJECT; - token = p.nextToken(); - assert token == XContentParser.Token.END_OBJECT; - return SupportedContext.valueOf(contextType.toUpperCase(Locale.ROOT)); - }, CONTEXT_FIELD); + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), CONTEXT_FIELD); + PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), ContextSetup::parse, CONTEXT_SETUP_FIELD); + } + + static final Map> SUPPORTED_CONTEXTS; + + static { + Map> supportedContexts = new HashMap<>(); + supportedContexts.put("painless_test", PainlessTestScript.CONTEXT); + supportedContexts.put("filter", FilterScript.CONTEXT); + supportedContexts.put("score", ScoreScript.CONTEXT); + SUPPORTED_CONTEXTS = Collections.unmodifiableMap(supportedContexts); + } + + static ScriptContext fromScriptContextName(String name) { + ScriptContext scriptContext = SUPPORTED_CONTEXTS.get(name); + if (scriptContext == null) { + throw new UnsupportedOperationException("unsupported script context name [" + name + "]"); + } + return scriptContext; + } + + static class ContextSetup implements Writeable, ToXContentObject { + + private static final ParseField INDEX_FIELD = new ParseField("index"); + private static final ParseField DOCUMENT_FIELD = new ParseField("document"); + private static final ParseField QUERY_FIELD = new ParseField("query"); + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("execute_script_context", + args -> new ContextSetup((String) args[0], (BytesReference) args[1], (QueryBuilder) args[2])); + + static { + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), INDEX_FIELD); + PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), (p, c) -> { + try (XContentBuilder b = XContentBuilder.builder(p.contentType().xContent())) { + b.copyCurrentStructure(p); + return BytesReference.bytes(b); + } + }, DOCUMENT_FIELD); + PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), (p, c) -> + AbstractQueryBuilder.parseInnerQueryBuilder(p), QUERY_FIELD); + } + + private final String index; + private final BytesReference document; + private final QueryBuilder query; + + private XContentType xContentType; + + static ContextSetup parse(XContentParser parser, Void context) throws IOException { + ContextSetup contextSetup = PARSER.parse(parser, null); + contextSetup.setXContentType(parser.contentType()); + return contextSetup; + } + + ContextSetup(String index, BytesReference document, QueryBuilder query) { + this.index = index; + this.document = document; + this.query = query; + } + + ContextSetup(StreamInput in) throws IOException { + index = in.readOptionalString(); + document = in.readOptionalBytesReference(); + String xContentType = in.readOptionalString(); + if (xContentType != null) { + this.xContentType = XContentType.fromMediaType(xContentType); + } + query = in.readOptionalNamedWriteable(QueryBuilder.class); + } + + public String getIndex() { + return index; + } + + public BytesReference getDocument() { + return document; + } + + public QueryBuilder getQuery() { + return query; + } + + public XContentType getXContentType() { + return xContentType; + } + + public void setXContentType(XContentType xContentType) { + this.xContentType = xContentType; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ContextSetup that = (ContextSetup) o; + return Objects.equals(index, that.index) && + Objects.equals(document, that.document) && + Objects.equals(query, that.query); + } + + @Override + public int hashCode() { + return Objects.hash(index, document, query); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalString(index); + out.writeOptionalBytesReference(document); + out.writeOptionalString(xContentType != null ? xContentType.mediaType(): null); + out.writeOptionalNamedWriteable(query); + } + + @Override + public String toString() { + return "ContextSetup{" + + ", index='" + index + '\'' + + ", document=" + document + + ", query=" + query + + ", xContentType=" + xContentType + + '}'; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + { + if (index != null) { + builder.field(INDEX_FIELD.getPreferredName(), index); + } + if (document != null) { + builder.field(DOCUMENT_FIELD.getPreferredName()); + try (XContentParser parser = XContentHelper.createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, document, xContentType)) { + builder.generator().copyCurrentStructure(parser); + } + } + if (query != null) { + builder.field(QUERY_FIELD.getPreferredName(), query); + } + } + builder.endObject(); + return builder; + } + } private Script script; - private SupportedContext context; + private ScriptContext context = PainlessTestScript.CONTEXT; + private ContextSetup contextSetup; static Request parse(XContentParser parser) throws IOException { return PARSER.parse(parser, null); } - Request(Script script, SupportedContext context) { + Request(Script script, String scriptContextName, ContextSetup setup) { this.script = Objects.requireNonNull(script); - this.context = context != null ? context : SupportedContext.PAINLESS_TEST; + if (scriptContextName != null) { + this.context = fromScriptContextName(scriptContextName); + } + if (setup != null) { + this.contextSetup = setup; + index(contextSetup.index); + } } Request() { @@ -118,16 +293,28 @@ public Script getScript() { return script; } - public SupportedContext getContext() { + public ScriptContext getContext() { return context; } + public ContextSetup getContextSetup() { + return contextSetup; + } + @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; if (script.getType() != ScriptType.INLINE) { validationException = addValidationError("only inline scripts are supported", validationException); } + if (needDocumentAndIndex(context)) { + if (contextSetup.index == null) { + validationException = addValidationError("index is a required parameter for current context", validationException); + } + if (contextSetup.document == null) { + validationException = addValidationError("document is a required parameter for current context", validationException); + } + } return validationException; } @@ -135,26 +322,35 @@ public ActionRequestValidationException validate() { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); script = new Script(in); - context = SupportedContext.fromId(in.readByte()); + if (in.getVersion().onOrBefore(Version.V_6_4_0)) { + byte scriptContextId = in.readByte(); + assert scriptContextId == 0; + } else { + context = fromScriptContextName(in.readString()); + contextSetup = in.readOptionalWriteable(ContextSetup::new); + } } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); script.writeTo(out); - out.writeByte(context.id); + if (out.getVersion().onOrBefore(Version.V_6_4_0)) { + out.writeByte((byte) 0); + } else { + out.writeString(context.name); + out.writeOptionalWriteable(contextSetup); + } } // For testing only: @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.field(SCRIPT_FIELD.getPreferredName(), script); - builder.startObject(CONTEXT_FIELD.getPreferredName()); - { - builder.startObject(context.name()); - builder.endObject(); + builder.field(CONTEXT_FIELD.getPreferredName(), context.name); + if (contextSetup != null) { + builder.field(CONTEXT_SETUP_FIELD.getPreferredName(), contextSetup); } - builder.endObject(); return builder; } @@ -164,41 +360,28 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; Request request = (Request) o; return Objects.equals(script, request.script) && - context == request.context; + Objects.equals(context, request.context) && + Objects.equals(contextSetup, request.contextSetup); } @Override public int hashCode() { - return Objects.hash(script, context); + return Objects.hash(script, context, contextSetup); } - public enum SupportedContext { - - PAINLESS_TEST((byte) 0); - - private final byte id; - - SupportedContext(byte id) { - this.id = id; - } - - public static SupportedContext fromId(byte id) { - switch (id) { - case 0: - return PAINLESS_TEST; - default: - throw new IllegalArgumentException("unknown context [" + id + "]"); - } - } + @Override + public String toString() { + return "Request{" + + "script=" + script + + "context=" + context + + ", contextSetup=" + contextSetup + + '}'; } - } - - public static class RequestBuilder extends ActionRequestBuilder { - - RequestBuilder(ElasticsearchClient client) { - super(client, INSTANCE, new Request()); + static boolean needDocumentAndIndex(ScriptContext scriptContext) { + return scriptContext == FilterScript.CONTEXT || scriptContext == ScoreScript.CONTEXT; } + } public static class Response extends ActionResponse implements ToXContentObject { @@ -274,31 +457,139 @@ public interface Factory { } - public static class TransportAction extends HandledTransportAction { - + public static class TransportAction extends TransportSingleShardAction { private final ScriptService scriptService; + private final IndicesService indicesServices; @Inject - public TransportAction(Settings settings, TransportService transportService, - ActionFilters actionFilters, ScriptService scriptService) { - super(settings, NAME, transportService, actionFilters, Request::new); + public TransportAction(Settings settings, ThreadPool threadPool, TransportService transportService, + ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, + ScriptService scriptService, ClusterService clusterService, IndicesService indicesServices) { + super(settings, NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, + // Forking a thread here, because only light weight operations should happen on network thread and + // Creating a in-memory index is not light weight + // TODO: is MANAGEMENT TP the right TP? Right now this is an admin api (see action name). + Request::new, ThreadPool.Names.MANAGEMENT); this.scriptService = scriptService; + this.indicesServices = indicesServices; } + @Override - protected void doExecute(Task task, Request request, ActionListener listener) { - switch (request.context) { - case PAINLESS_TEST: - PainlessTestScript.Factory factory = scriptService.compile(request.script, PainlessTestScript.CONTEXT); - PainlessTestScript painlessTestScript = factory.newInstance(request.script.getParams()); - String result = Objects.toString(painlessTestScript.execute()); - listener.onResponse(new Response(result)); - break; - default: - throw new UnsupportedOperationException("unsupported context [" + request.context + "]"); + protected Response newResponse() { + return new Response(); + } + + @Override + protected ClusterBlockException checkRequestBlock(ClusterState state, InternalRequest request) { + if (request.concreteIndex() != null) { + return super.checkRequestBlock(state, request); } + return null; + } + + @Override + protected boolean resolveIndex(Request request) { + return request.contextSetup != null && request.contextSetup.getIndex() != null; } + @Override + protected ShardsIterator shards(ClusterState state, InternalRequest request) { + if (request.concreteIndex() == null) { + return null; + } + return state.routingTable().index(request.concreteIndex()).randomAllActiveShardsIt(); + } + + @Override + protected Response shardOperation(Request request, ShardId shardId) throws IOException { + IndexService indexService; + if (request.contextSetup != null && request.contextSetup.getIndex() != null) { + ClusterState clusterState = clusterService.state(); + IndicesOptions indicesOptions = IndicesOptions.strictSingleIndexNoExpandForbidClosed(); + String indexExpression = request.contextSetup.index; + Index[] concreteIndices = + indexNameExpressionResolver.concreteIndices(clusterState, indicesOptions, indexExpression); + if (concreteIndices.length != 1) { + throw new IllegalArgumentException("[" + indexExpression + "] does not resolve to a single index"); + } + Index concreteIndex = concreteIndices[0]; + indexService = indicesServices.indexServiceSafe(concreteIndex); + } else { + indexService = null; + } + return innerShardOperation(request, scriptService, indexService); + } + + static Response innerShardOperation(Request request, ScriptService scriptService, IndexService indexService) throws IOException { + final ScriptContext scriptContext = request.context; + if (scriptContext == PainlessTestScript.CONTEXT) { + PainlessTestScript.Factory factory = scriptService.compile(request.script, PainlessTestScript.CONTEXT); + PainlessTestScript painlessTestScript = factory.newInstance(request.script.getParams()); + String result = Objects.toString(painlessTestScript.execute()); + return new Response(result); + } else if (scriptContext == FilterScript.CONTEXT) { + return prepareRamIndex(request, (context, leafReaderContext) -> { + FilterScript.Factory factory = scriptService.compile(request.script, FilterScript.CONTEXT); + FilterScript.LeafFactory leafFactory = + factory.newFactory(request.getScript().getParams(), context.lookup()); + FilterScript filterScript = leafFactory.newInstance(leafReaderContext); + filterScript.setDocument(0); + boolean result = filterScript.execute(); + return new Response(result); + }, indexService); + } else if (scriptContext == ScoreScript.CONTEXT) { + return prepareRamIndex(request, (context, leafReaderContext) -> { + ScoreScript.Factory factory = scriptService.compile(request.script, ScoreScript.CONTEXT); + ScoreScript.LeafFactory leafFactory = + factory.newFactory(request.getScript().getParams(), context.lookup()); + ScoreScript scoreScript = leafFactory.newInstance(leafReaderContext); + scoreScript.setDocument(0); + + if (request.contextSetup.query != null) { + Query luceneQuery = request.contextSetup.query.rewrite(context).toQuery(context); + IndexSearcher indexSearcher = new IndexSearcher(leafReaderContext.reader()); + luceneQuery = indexSearcher.rewrite(luceneQuery); + Weight weight = indexSearcher.createWeight(luceneQuery, true, 1f); + Scorer scorer = weight.scorer(indexSearcher.getIndexReader().leaves().get(0)); + // Consume the first (and only) match. + int docID = scorer.iterator().nextDoc(); + assert docID == scorer.docID(); + scoreScript.setScorer(scorer); + } + + double result = scoreScript.execute(); + return new Response(result); + }, indexService); + } else { + throw new UnsupportedOperationException("unsupported context [" + scriptContext.name + "]"); + } + } + + private static Response prepareRamIndex(Request request, + CheckedBiFunction handler, + IndexService indexService) throws IOException { + + Analyzer defaultAnalyzer = indexService.getIndexAnalyzers().getDefaultIndexAnalyzer(); + + try (RAMDirectory ramDirectory = new RAMDirectory()) { + try (IndexWriter indexWriter = new IndexWriter(ramDirectory, new IndexWriterConfig(defaultAnalyzer))) { + String index = indexService.index().getName(); + String type = indexService.mapperService().documentMapper().type(); + BytesReference document = request.contextSetup.document; + XContentType xContentType = request.contextSetup.xContentType; + SourceToParse sourceToParse = SourceToParse.source(index, type, "_id", document, xContentType); + ParsedDocument parsedDocument = indexService.mapperService().documentMapper().parse(sourceToParse); + indexWriter.addDocuments(parsedDocument.docs()); + try (IndexReader indexReader = DirectoryReader.open(indexWriter)) { + final long absoluteStartMillis = System.currentTimeMillis(); + QueryShardContext context = + indexService.newQueryShardContext(0, indexReader, () -> absoluteStartMillis, null); + return handler.apply(context, indexReader.leaves().get(0)); + } + } + } + } } static class RestAction extends BaseRestHandler { diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/PainlessExecuteApiTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/PainlessExecuteApiTests.java new file mode 100644 index 0000000000000..ce92a224f4e90 --- /dev/null +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/PainlessExecuteApiTests.java @@ -0,0 +1,113 @@ +/* + * 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.painless; + +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.query.MatchQueryBuilder; +import org.elasticsearch.painless.PainlessExecuteAction.Request; +import org.elasticsearch.painless.PainlessExecuteAction.Response; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptException; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.ScriptType; +import org.elasticsearch.test.ESSingleNodeTestCase; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static java.util.Collections.singletonMap; +import static org.elasticsearch.painless.PainlessExecuteAction.TransportAction.innerShardOperation; +import static org.hamcrest.Matchers.equalTo; + +public class PainlessExecuteApiTests extends ESSingleNodeTestCase { + + @Override + protected Collection> getPlugins() { + return Collections.singleton(PainlessPlugin.class); + } + + public void testDefaults() throws IOException { + ScriptService scriptService = getInstanceFromNode(ScriptService.class); + Request request = new Request(new Script("100.0 / 1000.0"), null, null); + Response response = innerShardOperation(request, scriptService, null); + assertThat(response.getResult(), equalTo("0.1")); + + Map params = new HashMap<>(); + params.put("count", 100.0D); + params.put("total", 1000.0D); + request = new Request(new Script(ScriptType.INLINE, "painless", "params.count / params.total", params), null, null); + response = innerShardOperation(request, scriptService, null); + assertThat(response.getResult(), equalTo("0.1")); + + Exception e = expectThrows(ScriptException.class, + () -> { + Request r = new Request(new Script(ScriptType.INLINE, + "painless", "params.count / params.total + doc['constant']", params), null, null); + innerShardOperation(r, scriptService, null); + }); + assertThat(e.getCause().getMessage(), equalTo("Variable [doc] is not defined.")); + } + + public void testFilterExecutionContext() throws IOException { + ScriptService scriptService = getInstanceFromNode(ScriptService.class); + IndexService indexService = createIndex("index", Settings.EMPTY, "doc", "field", "type=long"); + + Request.ContextSetup contextSetup = new Request.ContextSetup("index", new BytesArray("{\"field\": 3}"), null); + contextSetup.setXContentType(XContentType.JSON); + Request request = new Request(new Script("doc['field'].value >= 3"), "filter", contextSetup); + Response response = innerShardOperation(request, scriptService, indexService); + assertThat(response.getResult(), equalTo(true)); + + contextSetup = new Request.ContextSetup("index", new BytesArray("{\"field\": 3}"), null); + contextSetup.setXContentType(XContentType.JSON); + request = new Request(new Script(ScriptType.INLINE, "painless", "doc['field'].value >= params.max", + singletonMap("max", 3)), "filter", contextSetup); + response = innerShardOperation(request, scriptService, indexService); + assertThat(response.getResult(), equalTo(true)); + + contextSetup = new Request.ContextSetup("index", new BytesArray("{\"field\": 2}"), null); + contextSetup.setXContentType(XContentType.JSON); + request = new Request(new Script(ScriptType.INLINE, "painless", "doc['field'].value >= params.max", + singletonMap("max", 3)), "filter", contextSetup); + response = innerShardOperation(request, scriptService, indexService); + assertThat(response.getResult(), equalTo(false)); + } + + public void testScoreExecutionContext() throws IOException { + ScriptService scriptService = getInstanceFromNode(ScriptService.class); + IndexService indexService = createIndex("index", Settings.EMPTY, "doc", "rank", "type=long", "text", "type=text"); + + Request.ContextSetup contextSetup = new Request.ContextSetup("index", + new BytesArray("{\"rank\": 4.0, \"text\": \"quick brown fox\"}"), new MatchQueryBuilder("text", "fox")); + contextSetup.setXContentType(XContentType.JSON); + Request request = new Request(new Script(ScriptType.INLINE, "painless", + "Math.round((_score + (doc['rank'].value / params.max_rank)) * 100.0) / 100.0", singletonMap("max_rank", 5.0)), "score", + contextSetup); + Response response = innerShardOperation(request, scriptService, indexService); + assertThat(response.getResult(), equalTo(1.09D)); + } + +} diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/PainlessExecuteRequestTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/PainlessExecuteRequestTests.java index 488ae0e1643bc..44cd6b5304dc4 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/PainlessExecuteRequestTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/PainlessExecuteRequestTests.java @@ -18,9 +18,18 @@ */ package org.elasticsearch.painless; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.painless.PainlessExecuteAction.Request.ContextSetup; import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptType; +import org.elasticsearch.search.SearchModule; import org.elasticsearch.test.AbstractStreamableXContentTestCase; import java.io.IOException; @@ -28,12 +37,22 @@ public class PainlessExecuteRequestTests extends AbstractStreamableXContentTestCase { + @Override + protected NamedWriteableRegistry getNamedWriteableRegistry() { + return new NamedWriteableRegistry(new SearchModule(Settings.EMPTY, false, Collections.emptyList()).getNamedWriteables()); + } + + @Override + protected NamedXContentRegistry xContentRegistry() { + return new NamedXContentRegistry(new SearchModule(Settings.EMPTY, false, Collections.emptyList()).getNamedXContents()); + } + @Override protected PainlessExecuteAction.Request createTestInstance() { Script script = new Script(randomAlphaOfLength(10)); - PainlessExecuteAction.Request.SupportedContext context = randomBoolean() ? - PainlessExecuteAction.Request.SupportedContext.PAINLESS_TEST : null; - return new PainlessExecuteAction.Request(script, context); + ScriptContext context = randomBoolean() ? randomFrom(PainlessExecuteAction.Request.SUPPORTED_CONTEXTS.values()) : null; + ContextSetup contextSetup = randomBoolean() ? randomContextSetup() : null; + return new PainlessExecuteAction.Request(script, context != null ? context.name : null, contextSetup); } @Override @@ -53,9 +72,26 @@ protected boolean supportsUnknownFields() { public void testValidate() { Script script = new Script(ScriptType.STORED, null, randomAlphaOfLength(10), Collections.emptyMap()); - PainlessExecuteAction.Request request = new PainlessExecuteAction.Request(script, null); + PainlessExecuteAction.Request request = new PainlessExecuteAction.Request(script, null, null); Exception e = request.validate(); assertNotNull(e); assertEquals("Validation Failed: 1: only inline scripts are supported;", e.getMessage()); } + + private static ContextSetup randomContextSetup() { + String index = randomBoolean() ? randomAlphaOfLength(4) : null; + QueryBuilder query = randomBoolean() ? new MatchAllQueryBuilder() : null; + // TODO: pass down XContextType to createTestInstance() method. + // otherwise the document itself is different causing test failures. + // This should be done in a seperate change as the test instance is created before xcontent type is randomly picked and + // all the createTestInstance() methods need to be changed, which will make this a big chnage +// BytesReference doc = randomBoolean() ? new BytesArray("{}") : null; + BytesReference doc = null; + + ContextSetup contextSetup = new ContextSetup(index, doc, query); +// if (doc != null) { +// contextSetup.setXContentType(XContentType.JSON); +// } + return contextSetup; + } } diff --git a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/70_execute_painless_scripts.yml b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/70_execute_painless_scripts.yml index 7b915cc38dbc0..1e34a776189b8 100644 --- a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/70_execute_painless_scripts.yml +++ b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/70_execute_painless_scripts.yml @@ -1,3 +1,18 @@ +setup: + - do: + indices.create: + index: my-index + body: + mappings: + doc: + properties: + rank: + type: long + field: + type: keyword + text: + type: text + --- "Execute with defaults": - do: @@ -11,7 +26,7 @@ - match: { result: "0.1" } --- -"Execute with execute_api_script context": +"Execute with painless_test context": - do: scripts_painless_execute: body: @@ -20,6 +35,37 @@ params: var1: 10 var2: 100 - context: - painless_test: {} + context: "painless_test" - match: { result: "-90" } + +--- +"Execute with filter context": + - do: + scripts_painless_execute: + body: + script: + source: "doc['field'].value.length() <= params.max_length" + params: + max_length: 4 + context: "filter" + context_setup: + document: + field: "four" + index: "my-index" + - match: { result: true } + +--- +"Execute with score context": + - do: + scripts_painless_execute: + body: + script: + source: "doc['rank'].value / params.max_rank" + params: + max_rank: 5.0 + context: "score" + context_setup: + document: + rank: 4 + index: "my-index" + - match: { result: 0.8 }