diff --git a/server/src/main/java/org/elasticsearch/index/fielddata/SearchLookupAware.java b/server/src/main/java/org/elasticsearch/index/fielddata/SearchLookupAware.java new file mode 100644 index 0000000000000..a7b33090ca6f9 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/fielddata/SearchLookupAware.java @@ -0,0 +1,28 @@ +/* + * 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.index.fielddata; + +import org.elasticsearch.search.lookup.SearchLookup; + +//TODO this is a temporary interface only to avoid changing signature of MappedFieldType#fielddataBuilder +public interface SearchLookupAware { + + void setSearchLookup(SearchLookup searchLookup); +} diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index cc3a00bdd62a1..b550c6b5552b2 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -43,6 +43,7 @@ import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.SearchLookupAware; import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; @@ -208,7 +209,13 @@ public boolean allowExpensiveQueries() { @SuppressWarnings("unchecked") public > IFD getForField(MappedFieldType fieldType) { - return (IFD) indexFieldDataService.apply(fieldType, fullyQualifiedIndex.getName()); + IFD indexFieldData = (IFD) indexFieldDataService.apply(fieldType, fullyQualifiedIndex.getName()); + //TODO this is a temporary hack to inject search lookup to the scripted fielddata + // implementations without changing MappedFieldType#fielddataBuilder signature, as that would cause daily merge conflicts + if (indexFieldData instanceof SearchLookupAware) { + ((SearchLookupAware) indexFieldData).setSearchLookup(lookup()); + } + return indexFieldData; } public void addNamedQuery(String name, Query query) { diff --git a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/AbstractScriptFieldScript.java b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/AbstractScriptFieldScript.java index b9d605c796530..ec49d38ef0a30 100644 --- a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/AbstractScriptFieldScript.java +++ b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/AbstractScriptFieldScript.java @@ -9,9 +9,8 @@ import org.apache.lucene.index.LeafReaderContext; import org.elasticsearch.index.fielddata.ScriptDocValues; import org.elasticsearch.script.AggregationScript; -import org.elasticsearch.search.lookup.DocLookup; -import org.elasticsearch.search.lookup.LeafDocLookup; -import org.elasticsearch.search.lookup.SourceLookup; +import org.elasticsearch.search.lookup.LeafSearchLookup; +import org.elasticsearch.search.lookup.SearchLookup; import java.util.Map; @@ -21,23 +20,19 @@ */ public abstract class AbstractScriptFieldScript { private final Map params; - private final LeafReaderContext ctx; - private final SourceLookup source; - private final LeafDocLookup fieldData; + private final LeafSearchLookup leafSearchLookup; - public AbstractScriptFieldScript(Map params, SourceLookup source, DocLookup fieldData, LeafReaderContext ctx) { + public AbstractScriptFieldScript(Map params, SearchLookup searchLookup, LeafReaderContext ctx) { + this.leafSearchLookup = searchLookup.getLeafSearchLookup(ctx); + // TODO how do other scripts get stored fields exposed? Through asMap? I don't see any getters for them. this.params = params; - this.source = source; - this.fieldData = fieldData.getLeafDocLookup(ctx); - this.ctx = ctx; } /** * Set the document to run the script against. */ public final void setDocument(int docId) { - source.setSegmentAndDocument(ctx, docId); - fieldData.setDocument(docId); + this.leafSearchLookup.setDocument(docId); } /** @@ -51,16 +46,15 @@ public final Map getParams() { * Expose the {@code _source} to the script. */ public final Map getSource() { - return source; + return leafSearchLookup.source(); } /** * Expose field data to the script as {@code doc}. */ public final Map> getDoc() { - return fieldData; + return leafSearchLookup.doc(); } public abstract void execute(); - } diff --git a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/DoubleScriptFieldScript.java b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/DoubleScriptFieldScript.java index e86bfb20f71b6..8b66738b8bc74 100644 --- a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/DoubleScriptFieldScript.java +++ b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/DoubleScriptFieldScript.java @@ -11,8 +11,7 @@ import org.elasticsearch.painless.spi.WhitelistLoader; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptFactory; -import org.elasticsearch.search.lookup.DocLookup; -import org.elasticsearch.search.lookup.SourceLookup; +import org.elasticsearch.search.lookup.SearchLookup; import java.io.IOException; import java.util.List; @@ -29,7 +28,7 @@ static List whitelist() { public static final String[] PARAMETERS = {}; public interface Factory extends ScriptFactory { - LeafFactory newFactory(Map params, SourceLookup source, DocLookup fieldData); + LeafFactory newFactory(Map params, SearchLookup searchLookup); } public interface LeafFactory { @@ -38,14 +37,8 @@ public interface LeafFactory { private final DoubleConsumer sync; - public DoubleScriptFieldScript( - Map params, - SourceLookup source, - DocLookup fieldData, - LeafReaderContext ctx, - DoubleConsumer sync - ) { - super(params, source, fieldData, ctx); + public DoubleScriptFieldScript(Map params, SearchLookup searchLookup, LeafReaderContext ctx, DoubleConsumer sync) { + super(params, searchLookup, ctx); this.sync = sync; } diff --git a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/LongScriptFieldScript.java b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/LongScriptFieldScript.java index e99fa4a944ba3..9850cab424642 100644 --- a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/LongScriptFieldScript.java +++ b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/LongScriptFieldScript.java @@ -11,8 +11,7 @@ import org.elasticsearch.painless.spi.WhitelistLoader; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptFactory; -import org.elasticsearch.search.lookup.DocLookup; -import org.elasticsearch.search.lookup.SourceLookup; +import org.elasticsearch.search.lookup.SearchLookup; import java.io.IOException; import java.util.List; @@ -29,7 +28,7 @@ static List whitelist() { public static final String[] PARAMETERS = {}; public interface Factory extends ScriptFactory { - LeafFactory newFactory(Map params, SourceLookup source, DocLookup fieldData); + LeafFactory newFactory(Map params, SearchLookup searchLookup); } public interface LeafFactory { @@ -38,14 +37,8 @@ public interface LeafFactory { private final LongConsumer sync; - public LongScriptFieldScript( - Map params, - SourceLookup source, - DocLookup fieldData, - LeafReaderContext ctx, - LongConsumer sync - ) { - super(params, source, fieldData, ctx); + public LongScriptFieldScript(Map params, SearchLookup searchLookup, LeafReaderContext ctx, LongConsumer sync) { + super(params, searchLookup, ctx); this.sync = sync; } diff --git a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/RuntimeFields.java b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/RuntimeFields.java index 6b97b0dcb795c..f41c7efd44e69 100644 --- a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/RuntimeFields.java +++ b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/RuntimeFields.java @@ -6,25 +6,60 @@ package org.elasticsearch.xpack.runtimefields; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.env.Environment; +import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.plugins.MapperPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.ScriptPlugin; +import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.script.ScriptContext; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.watcher.ResourceWatcherService; +import org.elasticsearch.xpack.runtimefields.mapper.ScriptFieldMapper; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.function.Supplier; public final class RuntimeFields extends Plugin implements MapperPlugin, ScriptPlugin { + private final ScriptFieldMapper.TypeParser scriptTypeParser = new ScriptFieldMapper.TypeParser(); + @Override public Map getMappers() { - return Collections.emptyMap(); + return Collections.singletonMap(ScriptFieldMapper.CONTENT_TYPE, scriptTypeParser); } @Override public List> getContexts() { return List.of(DoubleScriptFieldScript.CONTEXT, LongScriptFieldScript.CONTEXT, StringScriptFieldScript.CONTEXT); } + + @Override + public Collection createComponents( + Client client, + ClusterService clusterService, + ThreadPool threadPool, + ResourceWatcherService resourceWatcherService, + ScriptService scriptService, + NamedXContentRegistry xContentRegistry, + Environment environment, + NodeEnvironment nodeEnvironment, + NamedWriteableRegistry namedWriteableRegistry, + IndexNameExpressionResolver indexNameExpressionResolver, + Supplier repositoriesServiceSupplier + ) { + // looks like createComponents gets called after getMappers + this.scriptTypeParser.setScriptService(scriptService); + return Collections.emptyList(); + } } diff --git a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/StringScriptFieldScript.java b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/StringScriptFieldScript.java index 2556381981236..391be84b8d964 100644 --- a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/StringScriptFieldScript.java +++ b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/StringScriptFieldScript.java @@ -11,8 +11,7 @@ import org.elasticsearch.painless.spi.WhitelistLoader; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptFactory; -import org.elasticsearch.search.lookup.DocLookup; -import org.elasticsearch.search.lookup.SourceLookup; +import org.elasticsearch.search.lookup.SearchLookup; import java.io.IOException; import java.util.List; @@ -20,7 +19,7 @@ import java.util.function.Consumer; public abstract class StringScriptFieldScript extends AbstractScriptFieldScript { - static final ScriptContext CONTEXT = new ScriptContext<>("string_script_field", Factory.class); + public static final ScriptContext CONTEXT = new ScriptContext<>("string_script_field", Factory.class); static List whitelist() { return List.of(WhitelistLoader.loadFromResourceFiles(RuntimeFieldsPainlessExtension.class, "string_whitelist.txt")); @@ -29,7 +28,7 @@ static List whitelist() { public static final String[] PARAMETERS = {}; public interface Factory extends ScriptFactory { - LeafFactory newFactory(Map params, SourceLookup source, DocLookup fieldData); + LeafFactory newFactory(Map params, SearchLookup searchLookup); } public interface LeafFactory { @@ -38,14 +37,8 @@ public interface LeafFactory { private final Consumer sync; - public StringScriptFieldScript( - Map params, - SourceLookup source, - DocLookup fieldData, - LeafReaderContext ctx, - Consumer sync - ) { - super(params, source, fieldData, ctx); + public StringScriptFieldScript(Map params, SearchLookup searchLookup, LeafReaderContext ctx, Consumer sync) { + super(params, searchLookup, ctx); this.sync = sync; } diff --git a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/fielddata/ScriptBinaryDocValues.java b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/fielddata/ScriptBinaryDocValues.java new file mode 100644 index 0000000000000..ae0edeae46b4e --- /dev/null +++ b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/fielddata/ScriptBinaryDocValues.java @@ -0,0 +1,41 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.runtimefields.fielddata; + +import org.elasticsearch.index.fielddata.SortingBinaryDocValues; +import org.elasticsearch.xpack.runtimefields.StringScriptFieldScript; + +public final class ScriptBinaryDocValues extends SortingBinaryDocValues { + + private final StringScriptFieldScript script; + private final ScriptBinaryFieldData.ScriptBinaryResult scriptBinaryResult; + + ScriptBinaryDocValues(StringScriptFieldScript script, ScriptBinaryFieldData.ScriptBinaryResult scriptBinaryResult) { + this.script = script; + this.scriptBinaryResult = scriptBinaryResult; + } + + @Override + public boolean advanceExact(int doc) { + script.setDocument(doc); + script.execute(); + + count = scriptBinaryResult.getResult().size(); + if (count == 0) { + grow(); + return false; + } + + int i = 0; + for (String value : scriptBinaryResult.getResult()) { + grow(); + values[i++].copyChars(value); + } + sort(); + return true; + } +} diff --git a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/fielddata/ScriptBinaryFieldData.java b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/fielddata/ScriptBinaryFieldData.java new file mode 100644 index 0000000000000..42ca35731274b --- /dev/null +++ b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/fielddata/ScriptBinaryFieldData.java @@ -0,0 +1,175 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.runtimefields.fielddata; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.SortField; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.index.AbstractIndexComponent; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.IndexFieldDataCache; +import org.elasticsearch.index.fielddata.LeafFieldData; +import org.elasticsearch.index.fielddata.ScriptDocValues; +import org.elasticsearch.index.fielddata.SearchLookupAware; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.indices.breaker.CircuitBreakerService; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.MultiValueMode; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; +import org.elasticsearch.search.lookup.SearchLookup; +import org.elasticsearch.search.sort.BucketedSort; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.xpack.runtimefields.StringScriptFieldScript; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public final class ScriptBinaryFieldData extends AbstractIndexComponent + implements + IndexFieldData, + SearchLookupAware { + + public static class Builder implements IndexFieldData.Builder { + + private final StringScriptFieldScript.Factory scriptFactory; + + public Builder(StringScriptFieldScript.Factory scriptFactory) { + this.scriptFactory = scriptFactory; + } + + @Override + public IndexFieldData build( + IndexSettings indexSettings, + MappedFieldType fieldType, + IndexFieldDataCache cache, + CircuitBreakerService breakerService, + MapperService mapperService + ) { + return new ScriptBinaryFieldData(indexSettings, fieldType.name(), scriptFactory); + } + } + + private final String fieldName; + private final StringScriptFieldScript.Factory scriptFactory; + private final SetOnce leafFactory = new SetOnce<>(); + + private ScriptBinaryFieldData(IndexSettings indexSettings, String fieldName, StringScriptFieldScript.Factory scriptFactory) { + super(indexSettings); + this.fieldName = fieldName; + this.scriptFactory = scriptFactory; + } + + public void setSearchLookup(SearchLookup searchLookup) { + // TODO wire the params from the mappings definition, we don't parse them yet + this.leafFactory.set(scriptFactory.newFactory(Collections.emptyMap(), searchLookup)); + } + + @Override + public String getFieldName() { + return fieldName; + } + + @Override + public ValuesSourceType getValuesSourceType() { + return CoreValuesSourceType.BYTES; + } + + @Override + public ScriptBinaryLeafFieldData load(LeafReaderContext context) { + try { + return loadDirect(context); + } catch (Exception e) { + if (e instanceof ElasticsearchException) { + throw (ElasticsearchException) e; + } else { + throw new ElasticsearchException(e); + } + } + } + + @Override + public ScriptBinaryLeafFieldData loadDirect(LeafReaderContext context) throws IOException { + ScriptBinaryResult scriptBinaryResult = new ScriptBinaryResult(); + return new ScriptBinaryLeafFieldData( + new ScriptBinaryDocValues(leafFactory.get().newInstance(context, scriptBinaryResult::accept), scriptBinaryResult) + ); + } + + @Override + public SortField sortField(Object missingValue, MultiValueMode sortMode, XFieldComparatorSource.Nested nested, boolean reverse) { + final XFieldComparatorSource source = new BytesRefFieldComparatorSource(this, missingValue, sortMode, nested); + return new SortField(getFieldName(), source, reverse); + } + + @Override + public BucketedSort newBucketedSort( + BigArrays bigArrays, + Object missingValue, + MultiValueMode sortMode, + XFieldComparatorSource.Nested nested, + SortOrder sortOrder, + DocValueFormat format, + int bucketSize, + BucketedSort.ExtraData extra + ) { + throw new IllegalArgumentException("only supported on numeric fields"); + } + + @Override + public void clear() { + + } + + static class ScriptBinaryLeafFieldData implements LeafFieldData { + private final ScriptBinaryDocValues scriptBinaryDocValues; + + ScriptBinaryLeafFieldData(ScriptBinaryDocValues scriptBinaryDocValues) { + this.scriptBinaryDocValues = scriptBinaryDocValues; + } + + @Override + public ScriptDocValues getScriptValues() { + return new ScriptDocValues.Strings(getBytesValues()); + } + + @Override + public SortedBinaryDocValues getBytesValues() { + return scriptBinaryDocValues; + } + + @Override + public long ramBytesUsed() { + return 0; + } + + @Override + public void close() { + + } + } + + static class ScriptBinaryResult { + private final List result = new ArrayList<>(); + + void accept(String value) { + this.result.add(value); + } + + List getResult() { + return result; + } + } +} diff --git a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/mapper/RuntimeKeywordMappedFieldType.java b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/mapper/RuntimeKeywordMappedFieldType.java new file mode 100644 index 0000000000000..5aef8e857408a --- /dev/null +++ b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/mapper/RuntimeKeywordMappedFieldType.java @@ -0,0 +1,85 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.runtimefields.mapper; + +import org.apache.lucene.search.Query; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.ToXContent.Params; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.TextSearchInfo; +import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.script.Script; +import org.elasticsearch.xpack.runtimefields.StringScriptFieldScript; +import org.elasticsearch.xpack.runtimefields.fielddata.ScriptBinaryFieldData; + +import java.io.IOException; +import java.util.Map; + +public final class RuntimeKeywordMappedFieldType extends MappedFieldType { + + private final Script script; + private final StringScriptFieldScript.Factory scriptFactory; + + RuntimeKeywordMappedFieldType(String name, Script script, StringScriptFieldScript.Factory scriptFactory, Map meta) { + super(name, false, false, TextSearchInfo.NONE, meta); + this.script = script; + this.scriptFactory = scriptFactory; + } + + RuntimeKeywordMappedFieldType(RuntimeKeywordMappedFieldType ref) { + super(ref); + this.script = ref.script; + this.scriptFactory = ref.scriptFactory; + } + + @Override + public MappedFieldType clone() { + return new RuntimeKeywordMappedFieldType(this); + } + + @Override + public Object valueForDisplay(Object value) { + if (value == null) { + return null; + } + // keywords are internally stored as utf8 bytes + BytesRef binaryValue = (BytesRef) value; + return binaryValue.utf8ToString(); + } + + @Override + public String typeName() { + // TODO not sure what we should return here: the runtime type or the field type? + // why is the same string returned from three different methods? + return ScriptFieldMapper.CONTENT_TYPE; + } + + @Override + public IndexFieldData.Builder fielddataBuilder(String fullyQualifiedIndexName) { + // TODO once we get SearchLookup as an argument, we can already call scriptFactory.newFactory here and pass through the result + return new ScriptBinaryFieldData.Builder(scriptFactory); + } + + @Override + public Query termQuery(Object value, QueryShardContext context) { + return null; + } + + @Override + public Query existsQuery(QueryShardContext context) { + return null; + } + + void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException { + builder.field("runtime_type", "keyword"); + builder.field("script", script.getIdOrCode()); // TODO For some reason this doesn't allow us to do the full xcontent of the script. + } + + // TODO do we need to override equals/hashcode? +} diff --git a/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/mapper/ScriptFieldMapper.java b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/mapper/ScriptFieldMapper.java new file mode 100644 index 0000000000000..c6fcd43635865 --- /dev/null +++ b/x-pack/plugin/runtime-fields/src/main/java/org/elasticsearch/xpack/runtimefields/mapper/ScriptFieldMapper.java @@ -0,0 +1,138 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.runtimefields.mapper; + +import org.apache.lucene.document.FieldType; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.index.mapper.BooleanFieldMapper; +import org.elasticsearch.index.mapper.FieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.Mapper; +import org.elasticsearch.index.mapper.MapperParsingException; +import org.elasticsearch.index.mapper.ParseContext; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.xpack.runtimefields.StringScriptFieldScript; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public final class ScriptFieldMapper extends FieldMapper { + + public static final String CONTENT_TYPE = "script"; + + private static final FieldType FIELD_TYPE = new FieldType(); + + ScriptFieldMapper(String simpleName, MappedFieldType mappedFieldType, MultiFields multiFields, CopyTo copyTo) { + super(simpleName, FIELD_TYPE, mappedFieldType, multiFields, copyTo); + } + + @Override + protected void parseCreateField(ParseContext context) { + // there is no field! + } + + @Override + protected void mergeOptions(FieldMapper other, List conflicts) { + // TODO implement this + } + + @Override + protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException { + super.doXContentBody(builder, includeDefaults, params); + RuntimeKeywordMappedFieldType fieldType = (RuntimeKeywordMappedFieldType) fieldType(); + fieldType.doXContentBody(builder, includeDefaults, params); + } + + @Override + protected String contentType() { + return CONTENT_TYPE; + } + + public static class Builder extends FieldMapper.Builder { + + private final ScriptService scriptService; + + private String runtimeType; + private Script script; + + protected Builder(String name, ScriptService scriptService) { + super(name, FIELD_TYPE); + this.scriptService = scriptService; + } + + public void runtimeType(String runtimeType) { + this.runtimeType = runtimeType; + } + + public void script(Script script) { + this.script = script; + } + + @Override + public ScriptFieldMapper build(BuilderContext context) { + if (runtimeType == null) { + throw new IllegalArgumentException("runtime_type must be specified"); + } + if (script == null) { + throw new IllegalArgumentException("script must be specified"); + } + + MappedFieldType mappedFieldType; + if (runtimeType.equals("keyword")) { + StringScriptFieldScript.Factory factory = scriptService.compile(script, StringScriptFieldScript.CONTEXT); + mappedFieldType = new RuntimeKeywordMappedFieldType(buildFullName(context), script, factory, meta); + } else { + throw new IllegalArgumentException("runtime_type [" + runtimeType + "] not supported"); + } + // TODO copy to and multi_fields... not sure what needs to be done. + return new ScriptFieldMapper(name, mappedFieldType, multiFieldsBuilder.build(this, context), copyTo); + } + } + + public static class TypeParser implements Mapper.TypeParser { + + private final SetOnce scriptService = new SetOnce<>(); + + public void setScriptService(ScriptService scriptService) { + this.scriptService.set(scriptService); + } + + @Override + public ScriptFieldMapper.Builder parse(String name, Map node, ParserContext parserContext) + throws MapperParsingException { + Builder builder = new Builder(name, scriptService.get()); + for (Iterator> iterator = node.entrySet().iterator(); iterator.hasNext();) { + Map.Entry entry = iterator.next(); + String propName = entry.getKey(); + Object propNode = entry.getValue(); + if (propName.equals("runtime_type")) { + if (propNode == null) { + throw new MapperParsingException("Property [runtime_type] cannot be null."); + } + builder.runtimeType(XContentMapValues.nodeStringValue(propNode, name + ".runtime_type")); + iterator.remove(); + } else if (propName.equals("script")) { + if (propNode == null) { + throw new MapperParsingException("Property [script] cannot be null."); + } + // TODO this should become an object and support the usual script syntax, including lang and params + builder.script(new Script(XContentMapValues.nodeStringValue(propNode, name + ".script"))); + iterator.remove(); + } + } + // TODO these get passed in sometimes and we don't need them + node.remove("doc_values"); + node.remove("index"); + return builder; + } + } +} diff --git a/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/double_whitelist.txt b/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/double_whitelist.txt index 55a177f21579c..ce8963b6f7cbf 100644 --- a/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/double_whitelist.txt +++ b/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/double_whitelist.txt @@ -4,7 +4,7 @@ # you may not use this file except in compliance with the Elastic License. # -# The whitelist for long-valued runtime fields +# The whitelist for double-valued runtime fields class org.elasticsearch.xpack.runtimefields.DoubleScriptFieldScript @no_import { } @@ -12,3 +12,7 @@ class org.elasticsearch.xpack.runtimefields.DoubleScriptFieldScript @no_import { static_import { void value(org.elasticsearch.xpack.runtimefields.DoubleScriptFieldScript, double) bound_to org.elasticsearch.xpack.runtimefields.DoubleScriptFieldScript$Value } + +# This import is required to make painless happy and it isn't 100% clear why +class org.elasticsearch.xpack.runtimefields.DoubleScriptFieldScript$Factory @no_import { +} diff --git a/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/long_whitelist.txt b/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/long_whitelist.txt index 630c679bc21cc..f5f3d0245f949 100644 --- a/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/long_whitelist.txt +++ b/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/long_whitelist.txt @@ -12,3 +12,7 @@ class org.elasticsearch.xpack.runtimefields.LongScriptFieldScript @no_import { static_import { void value(org.elasticsearch.xpack.runtimefields.LongScriptFieldScript, long) bound_to org.elasticsearch.xpack.runtimefields.LongScriptFieldScript$Value } + +# This import is required to make painless happy and it isn't 100% clear why +class org.elasticsearch.xpack.runtimefields.LongScriptFieldScript$Factory @no_import { +} diff --git a/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/string_whitelist.txt b/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/string_whitelist.txt index e0c4e367850f3..5570891fb4b19 100644 --- a/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/string_whitelist.txt +++ b/x-pack/plugin/runtime-fields/src/main/resources/org/elasticsearch/xpack/runtimefields/string_whitelist.txt @@ -4,7 +4,7 @@ # you may not use this file except in compliance with the Elastic License. # -# The whitelist for long-valued runtime fields +# The whitelist for string-valued runtime fields class org.elasticsearch.xpack.runtimefields.StringScriptFieldScript @no_import { } @@ -12,3 +12,7 @@ class org.elasticsearch.xpack.runtimefields.StringScriptFieldScript @no_import { static_import { void value(org.elasticsearch.xpack.runtimefields.StringScriptFieldScript, String) bound_to org.elasticsearch.xpack.runtimefields.StringScriptFieldScript$Value } + +# This import is required to make painless happy and it isn't 100% clear why +class org.elasticsearch.xpack.runtimefields.StringScriptFieldScript$Factory @no_import { +} diff --git a/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/DoubleScriptFieldScriptTests.java b/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/DoubleScriptFieldScriptTests.java index 37c6c5d35ce41..23081d1ff3427 100644 --- a/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/DoubleScriptFieldScriptTests.java +++ b/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/DoubleScriptFieldScriptTests.java @@ -16,8 +16,7 @@ import org.elasticsearch.index.mapper.NumberFieldMapper.NumberFieldType; import org.elasticsearch.index.mapper.NumberFieldMapper.NumberType; import org.elasticsearch.script.ScriptContext; -import org.elasticsearch.search.lookup.DocLookup; -import org.elasticsearch.search.lookup.SourceLookup; +import org.elasticsearch.search.lookup.SearchLookup; import java.io.IOException; import java.util.List; @@ -107,10 +106,9 @@ protected ScriptContext scriptContext() { protected DoubleScriptFieldScript.LeafFactory newLeafFactory( DoubleScriptFieldScript.Factory factory, Map params, - SourceLookup source, - DocLookup fieldData + SearchLookup searchLookup ) { - return factory.newFactory(params, source, fieldData); + return factory.newFactory(params, searchLookup); } @Override diff --git a/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/LongScriptFieldScriptTests.java b/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/LongScriptFieldScriptTests.java index 05d48808b5620..ef0c5cd304d0e 100644 --- a/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/LongScriptFieldScriptTests.java +++ b/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/LongScriptFieldScriptTests.java @@ -15,8 +15,7 @@ import org.elasticsearch.index.mapper.NumberFieldMapper.NumberFieldType; import org.elasticsearch.index.mapper.NumberFieldMapper.NumberType; import org.elasticsearch.script.ScriptContext; -import org.elasticsearch.search.lookup.DocLookup; -import org.elasticsearch.search.lookup.SourceLookup; +import org.elasticsearch.search.lookup.SearchLookup; import java.io.IOException; import java.util.List; @@ -93,10 +92,9 @@ protected ScriptContext scriptContext() { protected LongScriptFieldScript.LeafFactory newLeafFactory( LongScriptFieldScript.Factory factory, Map params, - SourceLookup source, - DocLookup fieldData + SearchLookup searchLookup ) { - return factory.newFactory(params, source, fieldData); + return factory.newFactory(params, searchLookup); } @Override diff --git a/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/ScriptFieldScriptTestCase.java b/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/ScriptFieldScriptTestCase.java index a086953142662..3b7a819d04b40 100644 --- a/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/ScriptFieldScriptTestCase.java +++ b/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/ScriptFieldScriptTestCase.java @@ -31,8 +31,7 @@ import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptModule; import org.elasticsearch.script.ScriptService; -import org.elasticsearch.search.lookup.DocLookup; -import org.elasticsearch.search.lookup.SourceLookup; +import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -50,7 +49,7 @@ public abstract class ScriptFieldScriptTestCase extends ESTestCase { protected abstract ScriptContext scriptContext(); - protected abstract LF newLeafFactory(F factory, Map params, SourceLookup source, DocLookup fieldData); + protected abstract LF newLeafFactory(F factory, Map params, SearchLookup searchLookup); protected abstract S newInstance(LF leafFactory, LeafReaderContext context, List results) throws IOException; @@ -67,14 +66,13 @@ public List loadExtensions(Class extensionPointType) { }); ScriptModule scriptModule = new ScriptModule(Settings.EMPTY, List.of(painlessPlugin, new RuntimeFields())); Map params = new HashMap<>(); - SourceLookup source = new SourceLookup(); MapperService mapperService = mock(MapperService.class); for (MappedFieldType type : types) { when(mapperService.fieldType(type.name())).thenReturn(type); } Function> fieldDataLookup = ft -> ft.fielddataBuilder("test") .build(indexSettings(), ft, null, new NoneCircuitBreakerService(), mapperService); - DocLookup fieldData = new DocLookup(mapperService, fieldDataLookup); + SearchLookup searchLookup = new SearchLookup(mapperService, fieldDataLookup); try (ScriptService scriptService = new ScriptService(Settings.EMPTY, scriptModule.engines, scriptModule.contexts)) { F factory = AccessController.doPrivileged( (PrivilegedAction) () -> scriptService.compile(new Script(script), scriptContext()) @@ -84,7 +82,7 @@ public List loadExtensions(Class extensionPointType) { indexBuilder.accept(indexWriter); try (DirectoryReader reader = indexWriter.getReader()) { IndexSearcher searcher = newSearcher(reader); - LF leafFactory = newLeafFactory(factory, params, source, fieldData); + LF leafFactory = newLeafFactory(factory, params, searchLookup); List result = new ArrayList<>(); searcher.search(new MatchAllDocsQuery(), new Collector() { @Override @@ -97,10 +95,10 @@ public LeafCollector getLeafCollector(LeafReaderContext context) throws IOExcept S compiled = newInstance(leafFactory, context, result); return new LeafCollector() { @Override - public void setScorer(Scorable scorer) throws IOException {} + public void setScorer(Scorable scorer) {} @Override - public void collect(int doc) throws IOException { + public void collect(int doc) { compiled.setDocument(doc); compiled.execute(); } diff --git a/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/StringScriptFieldScriptTests.java b/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/StringScriptFieldScriptTests.java index 658001f743281..5ecedd1762647 100644 --- a/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/StringScriptFieldScriptTests.java +++ b/x-pack/plugin/runtime-fields/src/test/java/org/elasticsearch/xpack/runtimefields/StringScriptFieldScriptTests.java @@ -14,8 +14,7 @@ import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.index.mapper.KeywordFieldMapper.KeywordFieldType; import org.elasticsearch.script.ScriptContext; -import org.elasticsearch.search.lookup.DocLookup; -import org.elasticsearch.search.lookup.SourceLookup; +import org.elasticsearch.search.lookup.SearchLookup; import java.io.IOException; import java.util.List; @@ -99,10 +98,9 @@ protected ScriptContext scriptContext() { protected StringScriptFieldScript.LeafFactory newLeafFactory( StringScriptFieldScript.Factory factory, Map params, - SourceLookup source, - DocLookup fieldData + SearchLookup searchLookup ) { - return factory.newFactory(params, source, fieldData); + return factory.newFactory(params, searchLookup); } @Override