add extra safety when accessing scripts or now and reqeusts are cached

This commit is contained in:
Simon Willnauer 2016-10-05 09:41:48 +02:00
parent 80bf44f709
commit 587bdcef38
43 changed files with 229 additions and 237 deletions

View File

@ -89,6 +89,7 @@ import java.util.Set;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.LongSupplier;
import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableMap;
@ -448,13 +449,13 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
* Creates a new QueryShardContext. The context has not types set yet, if types are required set them via
* {@link QueryShardContext#setTypes(String...)}
*/
public QueryShardContext newQueryShardContext(IndexReader indexReader) {
public QueryShardContext newQueryShardContext(IndexReader indexReader, LongSupplier nowInMillis) {
return new QueryShardContext(
indexSettings, indexCache.bitsetFilterCache(), indexFieldData, mapperService(),
similarityService(), nodeServicesProvider.getScriptService(), nodeServicesProvider.getIndicesQueriesRegistry(),
nodeServicesProvider.getClient(), indexReader,
nodeServicesProvider.getClusterService().state()
);
nodeServicesProvider.getClusterService().state(),
nowInMillis);
}
/**
@ -463,7 +464,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
* used for rewriting since it does not know about the current {@link IndexReader}.
*/
public QueryShardContext newQueryShardContext() {
return newQueryShardContext(null);
return newQueryShardContext(null, threadPool::estimatedTimeInMillis);
}
public ThreadPool getThreadPool() {

View File

@ -366,7 +366,7 @@ public class DateFieldMapper extends FieldMapper {
return () -> {
final SearchContext context = SearchContext.current();
return context != null
? context.nowInMillis()
? context.getQueryShardContext().nowInMillis()
: System.currentTimeMillis();
};
}

View File

@ -480,7 +480,7 @@ public class LegacyDateFieldMapper extends LegacyNumberFieldMapper {
public Long call() {
final SearchContext context = SearchContext.current();
return context != null
? context.nowInMillis()
? context.getQueryShardContext().nowInMillis()
: System.currentTimeMillis();
}
};

View File

@ -143,7 +143,7 @@ public class TTLFieldMapper extends MetadataFieldMapper {
long now;
SearchContext searchContext = SearchContext.current();
if (searchContext != null) {
now = searchContext.nowInMillis();
now = searchContext.getQueryShardContext().nowInMillis();
} else {
now = System.currentTimeMillis();
}

View File

@ -574,10 +574,9 @@ public final class InnerHitBuilder extends ToXContentToBytes implements Writeabl
innerHitsContext.docValueFieldsContext(new DocValueFieldsContext(docValueFields));
}
if (scriptFields != null) {
context.markAsNotCachable();
for (ScriptField field : scriptFields) {
SearchScript searchScript = innerHitsContext.getQueryShardContext().getScriptService().search(innerHitsContext.lookup(), field.script(),
ScriptContext.Standard.SEARCH, Collections.emptyMap());
SearchScript searchScript = innerHitsContext.getQueryShardContext().getSearchScript(field.script(),
ScriptContext.Standard.SEARCH, Collections.emptyMap());
innerHitsContext.scriptFields().add(new org.elasticsearch.search.fetch.subphase.ScriptFieldsContext.ScriptField(
field.fieldName(), searchScript, field.ignoreFailure()));
}

View File

@ -19,17 +19,24 @@
package org.elasticsearch.index.query;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParseFieldMatcherSupplier;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.script.ScriptSettings;
import java.util.Collections;
/**
* Context object used to rewrite {@link QueryBuilder} instances into simplified version.
*/
@ -42,6 +49,7 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier {
protected final IndexReader reader;
protected final ClusterState clusterState;
protected boolean cachable;
private final SetOnce<Boolean> executionMode = new SetOnce<>();
public QueryRewriteContext(IndexSettings indexSettings, MapperService mapperService, ScriptService scriptService,
IndicesQueriesRegistry indicesQueriesRegistry, Client client, IndexReader reader,
@ -70,13 +78,6 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier {
return indexSettings;
}
/**
* Returns a script service to fetch scripts.
*/
public final ScriptService getScriptService() {
return scriptService;
}
/**
* Return the MapperService.
*/
@ -118,11 +119,32 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier {
return new QueryParseContext(defaultScriptLanguage, indicesQueriesRegistry, parser, indexSettings.getParseFieldMatcher());
}
public void markAsNotCachable() {
protected final void markAsNotCachable() {
this.cachable = false;
}
public boolean isCachable() {
return cachable;
}
public void setCachabe(boolean cachabe) { this.cachable = cachabe; }
public BytesReference getTemplateBytes(Script template) {
failIfExecutionMode();
ExecutableScript executable = scriptService.executable(template,
ScriptContext.Standard.SEARCH, Collections.emptyMap());
return (BytesReference) executable.run();
}
public void setExecutionMode() {
this.executionMode.set(Boolean.TRUE);
}
protected void failIfExecutionMode() {
if (executionMode.get() == Boolean.TRUE) {
throw new IllegalArgumentException("features that prevent cachability are disabled on this context");
} else {
assert executionMode.get() == null : executionMode.get();
}
}
}

View File

@ -26,6 +26,9 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.function.Function;
import java.util.function.LongSupplier;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queryparser.classic.MapperQueryParser;
@ -54,7 +57,12 @@ import org.elasticsearch.index.mapper.TextFieldMapper;
import org.elasticsearch.index.query.support.NestedScope;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
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.SearchScript;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.lookup.SearchLookup;
@ -85,11 +93,12 @@ public class QueryShardContext extends QueryRewriteContext {
private boolean mapUnmappedFieldAsString;
private NestedScope nestedScope;
private boolean isFilter;
private final LongSupplier nowInMillis;
public QueryShardContext(IndexSettings indexSettings, BitsetFilterCache bitsetFilterCache, IndexFieldDataService indexFieldDataService,
MapperService mapperService, SimilarityService similarityService, ScriptService scriptService,
final IndicesQueriesRegistry indicesQueriesRegistry, Client client,
IndexReader reader, ClusterState clusterState) {
IndexReader reader, ClusterState clusterState, LongSupplier nowInMillis) {
super(indexSettings, mapperService, scriptService, indicesQueriesRegistry, client, reader, clusterState);
this.indexSettings = indexSettings;
this.similarityService = similarityService;
@ -99,12 +108,13 @@ public class QueryShardContext extends QueryRewriteContext {
this.allowUnmappedFields = indexSettings.isDefaultAllowUnmappedFields();
this.indicesQueriesRegistry = indicesQueriesRegistry;
this.nestedScope = new NestedScope();
this.nowInMillis = nowInMillis;
}
public QueryShardContext(QueryShardContext source) {
this(source.indexSettings, source.bitsetFilterCache, source.indexFieldDataService, source.mapperService,
source.similarityService, source.scriptService, source.indicesQueriesRegistry, source.client,
source.reader, source.clusterState);
source.reader, source.clusterState, source.nowInMillis);
this.types = source.getTypes();
}
@ -261,11 +271,8 @@ public class QueryShardContext extends QueryRewriteContext {
}
public long nowInMillis() {
SearchContext current = SearchContext.current();
if (current != null) {
return current.nowInMillis();
}
return System.currentTimeMillis();
failIfExecutionMode();
return nowInMillis.getAsLong();
}
public NestedScope nestedScope() {
@ -324,16 +331,33 @@ public class QueryShardContext extends QueryRewriteContext {
}
}
@Override
public void markAsNotCachable() {
super.markAsNotCachable();
SearchContext current = SearchContext.current();
if (current != null) {
current.markAsNotCachable();
}
}
public final Index index() {
return indexSettings.getIndex();
}
public SearchScript getSearchScript(Script script, ScriptContext context, Map<String, String> params) {
failIfExecutionMode();
markAsNotCachable();
return scriptService.search(lookup(), script, context, params);
}
public Function<Map<String, Object>, SearchScript> getLazySearchScript(Script script, ScriptContext context, Map<String, String> params) {
failIfExecutionMode();
markAsNotCachable();
CompiledScript compile = scriptService.compile(script, context, params);
return (p) -> scriptService.search(lookup(), compile, p);
}
public ExecutableScript getExecutableScript(Script script, ScriptContext context, Map<String, String> params) {
failIfExecutionMode();
markAsNotCachable();
return scriptService.executable(script, context, params);
}
public Function<Map<String, Object>, ExecutableScript> getLazyExecutableScript(Script script, ScriptContext context, Map<String, String> params) {
failIfExecutionMode();
markAsNotCachable();
CompiledScript executable = scriptService.compile(script, context, params);
return (p) -> scriptService.executable(executable, p);
}
}

View File

@ -133,8 +133,7 @@ public class ScriptQueryBuilder extends AbstractQueryBuilder<ScriptQueryBuilder>
@Override
protected Query doToQuery(QueryShardContext context) throws IOException {
context.markAsNotCachable();
return new ScriptQuery(script, context.getScriptService(), context.lookup());
return new ScriptQuery(script, context.getSearchScript(script, ScriptContext.Standard.SEARCH, Collections.emptyMap()));
}
static class ScriptQuery extends Query {
@ -143,9 +142,9 @@ public class ScriptQueryBuilder extends AbstractQueryBuilder<ScriptQueryBuilder>
private final SearchScript searchScript;
public ScriptQuery(Script script, ScriptService scriptService, SearchLookup searchLookup) {
public ScriptQuery(Script script, SearchScript searchScript) {
this.script = script;
this.searchScript = scriptService.search(searchLookup, script, ScriptContext.Standard.SEARCH, Collections.emptyMap());
this.searchScript = searchScript;
}
@Override

View File

@ -95,10 +95,8 @@ public class ScriptScoreFunctionBuilder extends ScoreFunctionBuilder<ScriptScore
@Override
protected ScoreFunction doToFunction(QueryShardContext context) {
context.markAsNotCachable();
try {
SearchScript searchScript = context.getScriptService().search(context.lookup(), script, ScriptContext.Standard.SEARCH,
Collections.emptyMap());
SearchScript searchScript = context.getSearchScript(script, ScriptContext.Standard.SEARCH, Collections.emptyMap());
return new ScriptScoreFunction(script, searchScript);
} catch (Exception e) {
throw new QueryShardException(context, "script_score: the script could not be loaded", e);

View File

@ -43,7 +43,6 @@ import java.util.Objects;
*/
public final class Script implements ToXContent, Writeable {
public static final ScriptType DEFAULT_TYPE = ScriptType.INLINE;
public static final String DEFAULT_SCRIPT_LANG = "painless";
private String script;

View File

@ -488,7 +488,11 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
*/
public SearchScript search(SearchLookup lookup, Script script, ScriptContext scriptContext, Map<String, String> params) {
CompiledScript compiledScript = compile(script, scriptContext, params);
return getScriptEngineServiceForLang(compiledScript.lang()).search(compiledScript, lookup, script.getParams());
return search(lookup, compiledScript, script.getParams());
}
public SearchScript search(SearchLookup lookup, CompiledScript compiledScript, Map<String, Object> params) {
return getScriptEngineServiceForLang(compiledScript.lang()).search(compiledScript, lookup, params);
}
private boolean isAnyScriptContextEnabled(String lang, ScriptType scriptType) {

View File

@ -51,7 +51,6 @@ import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.aggregations.SearchContextAggregations;
import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.fetch.FetchPhase;
@ -89,7 +88,6 @@ final class DefaultSearchContext extends SearchContext {
private final Counter timeEstimateCounter;
private SearchType searchType;
private final Engine.Searcher engineSearcher;
private final ScriptService scriptService;
private final BigArrays bigArrays;
private final IndexShard indexShard;
private final IndexService indexService;
@ -150,9 +148,9 @@ final class DefaultSearchContext extends SearchContext {
private FetchPhase fetchPhase;
DefaultSearchContext(long id, ShardSearchRequest request, SearchShardTarget shardTarget, Engine.Searcher engineSearcher,
IndexService indexService, IndexShard indexShard, ScriptService scriptService,
BigArrays bigArrays, Counter timeEstimateCounter, ParseFieldMatcher parseFieldMatcher, TimeValue timeout,
FetchPhase fetchPhase) {
IndexService indexService, IndexShard indexShard,
BigArrays bigArrays, Counter timeEstimateCounter, ParseFieldMatcher parseFieldMatcher, TimeValue timeout,
FetchPhase fetchPhase) {
super(parseFieldMatcher);
this.id = id;
this.request = request;
@ -160,7 +158,6 @@ final class DefaultSearchContext extends SearchContext {
this.searchType = request.searchType();
this.shardTarget = shardTarget;
this.engineSearcher = engineSearcher;
this.scriptService = scriptService;
// SearchContexts use a BigArrays that can circuit break
this.bigArrays = bigArrays.withCircuitBreaking();
this.dfsResult = new DfsSearchResult(id, shardTarget);
@ -171,7 +168,7 @@ final class DefaultSearchContext extends SearchContext {
this.searcher = new ContextIndexSearcher(engineSearcher, indexService.cache().query(), indexShard.getQueryCachingPolicy());
this.timeEstimateCounter = timeEstimateCounter;
this.timeout = timeout;
queryShardContext = indexService.newQueryShardContext(searcher.getIndexReader());
queryShardContext = indexService.newQueryShardContext(searcher.getIndexReader(), request::nowInMillis);
queryShardContext.setTypes(request.types());
}
@ -358,11 +355,6 @@ final class DefaultSearchContext extends SearchContext {
return originNanoTime;
}
@Override
protected long nowInMillisImpl() {
return request.nowInMillis();
}
@Override
public ScrollContext scrollContext() {
return this.scrollContext;
@ -501,11 +493,6 @@ final class DefaultSearchContext extends SearchContext {
return indexService.similarityService();
}
@Override
public ScriptService scriptService() {
return scriptService;
}
@Override
public BigArrays bigArrays() {
return bigArrays;

View File

@ -231,6 +231,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
*/
private void loadOrExecuteQueryPhase(final ShardSearchRequest request, final SearchContext context) throws Exception {
final boolean canCache = indicesService.canCache(request, context);
context.getQueryShardContext().setExecutionMode();
if (canCache) {
indicesService.loadIntoContext(request, context, queryPhase);
} else {
@ -568,7 +569,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
return new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, engineSearcher,
indexService,
indexShard, scriptService, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher,
indexShard, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher,
timeout, fetchPhase);
}

View File

@ -153,11 +153,11 @@ public class ExtendedBounds implements ToXContent, Writeable {
Long max = this.max;
assert format != null;
if (minAsStr != null) {
min = format.parseLong(minAsStr, false, context::nowInMillis);
min = format.parseLong(minAsStr, false, context.getQueryShardContext()::nowInMillis);
}
if (maxAsStr != null) {
// TODO: Should we rather pass roundUp=true?
max = format.parseLong(maxAsStr, false, context::nowInMillis);
max = format.parseLong(maxAsStr, false, context.getQueryShardContext()::nowInMillis);
}
if (min != null && max != null && min.compareTo(max) > 0) {
throw new SearchParseException(context, "[extended_bounds.min][" + min + "] cannot be greater than " +

View File

@ -119,10 +119,10 @@ public class RangeAggregator extends BucketsAggregator {
Double from = this.from;
Double to = this.to;
if (fromAsStr != null) {
from = parser.parseDouble(fromAsStr, false, context::nowInMillis);
from = parser.parseDouble(fromAsStr, false, context.getQueryShardContext()::nowInMillis);
}
if (toAsStr != null) {
to = parser.parseDouble(toAsStr, false, context::nowInMillis);
to = parser.parseDouble(toAsStr, false, context.getQueryShardContext()::nowInMillis);
}
return new Range(key, from, fromAsStr, to, toAsStr);
}

View File

@ -220,9 +220,7 @@ public class SignificantTermsAggregationBuilder extends ValuesSourceAggregationB
@Override
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(AggregationContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
if (significanceHeuristic.canCache() == false) {
context.searchContext().markAsNotCachable();
}
this.significanceHeuristic.initialize(context.searchContext());
return new SignificantTermsAggregatorFactory(name, type, config, includeExclude, executionHint, filterBuilder,
bucketCountThresholds, significanceHeuristic, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -91,7 +91,6 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
: searcher.count(filter);
this.bucketCountThresholds = bucketCountThresholds;
this.significanceHeuristic = significanceHeuristic;
this.significanceHeuristic.initialize(context.searchContext());
setFieldInfo();
}
@ -211,13 +210,13 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
}
}
assert execution != null;
DocValueFormat format = config.format();
if ((includeExclude != null) && (includeExclude.isRegexBased()) && format != DocValueFormat.RAW) {
throw new AggregationExecutionException("Aggregation [" + name + "] cannot support regular expression style include/exclude "
+ "settings as they can only be applied to string fields. Use an array of values for include/exclude clauses");
}
return execution.create(name, factories, valuesSource, format, bucketCountThresholds, includeExclude, context, parent,
significanceHeuristic, this, pipelineAggregators, metaData);
}

View File

@ -32,7 +32,6 @@ import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.Script.ScriptField;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.internal.SearchContext;
@ -49,7 +48,7 @@ public class ScriptHeuristic extends SignificanceHeuristic {
private final LongAccessor subsetDfHolder;
private final LongAccessor supersetDfHolder;
private final Script script;
ExecutableScript searchScript = null;
ExecutableScript executableScript = null;
public ScriptHeuristic(Script script) {
subsetSizeHolder = new LongAccessor();
@ -73,21 +72,20 @@ public class ScriptHeuristic extends SignificanceHeuristic {
@Override
public void initialize(InternalAggregation.ReduceContext context) {
initialize(context.scriptService());
initialize(context.scriptService().executable(script, ScriptContext.Standard.AGGS, Collections.emptyMap()));
}
@Override
public void initialize(SearchContext context) {
context.markAsNotCachable();
initialize(context.scriptService());
initialize(context.getQueryShardContext().getExecutableScript(script, ScriptContext.Standard.AGGS, Collections.emptyMap()));
}
public void initialize(ScriptService scriptService) {
searchScript = scriptService.executable(script, ScriptContext.Standard.AGGS, Collections.emptyMap());
searchScript.setNextVar("_subset_freq", subsetDfHolder);
searchScript.setNextVar("_subset_size", subsetSizeHolder);
searchScript.setNextVar("_superset_freq", supersetDfHolder);
searchScript.setNextVar("_superset_size", supersetSizeHolder);
public void initialize(ExecutableScript executableScript) {
this.executableScript = executableScript;
this.executableScript.setNextVar("_subset_freq", subsetDfHolder);
this.executableScript.setNextVar("_subset_size", subsetSizeHolder);
this.executableScript.setNextVar("_superset_freq", supersetDfHolder);
this.executableScript.setNextVar("_superset_size", supersetSizeHolder);
}
/**
@ -101,7 +99,7 @@ public class ScriptHeuristic extends SignificanceHeuristic {
*/
@Override
public double getScore(long subsetFreq, long subsetSize, long supersetFreq, long supersetSize) {
if (searchScript == null) {
if (executableScript == null) {
//In tests, wehn calling assertSearchResponse(..) the response is streamed one additional time with an arbitrary version, see assertVersionSerializable(..).
// Now, for version before 1.5.0 the score is computed after streaming the response but for scripts the script does not exists yet.
// assertSearchResponse() might therefore fail although there is no problem.
@ -113,7 +111,7 @@ public class ScriptHeuristic extends SignificanceHeuristic {
supersetSizeHolder.value = supersetSize;
subsetDfHolder.value = subsetFreq;
supersetDfHolder.value = supersetFreq;
return ((Number) searchScript.run()).doubleValue();
return ((Number) executableScript.run()).doubleValue();
}
@Override
@ -172,26 +170,6 @@ public class ScriptHeuristic extends SignificanceHeuristic {
return new ScriptHeuristic(script);
}
public static class ScriptHeuristicBuilder implements SignificanceHeuristicBuilder {
private Script script = null;
public ScriptHeuristicBuilder setScript(Script script) {
this.script = script;
return this;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params builderParams) throws IOException {
builder.startObject(NAME);
builder.field(ScriptField.SCRIPT.getPreferredName());
script.toXContent(builder, builderParams);
builder.endObject();
return builder;
}
}
public final class LongAccessor extends Number {
public long value;
@Override
@ -218,10 +196,5 @@ public class ScriptHeuristic extends SignificanceHeuristic {
return Long.toString(value);
}
}
@Override
public boolean canCache() {
return false;
}
}

View File

@ -57,8 +57,4 @@ public abstract class SignificanceHeuristic implements NamedWriteable, ToXConten
public void initialize(SearchContext context) {
}
public boolean canCache() {
return true;
}
}

View File

@ -26,18 +26,30 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.script.CompiledScript;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.Function;
public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder<ScriptedMetricAggregationBuilder> {
@ -182,11 +194,29 @@ public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder
@Override
protected ScriptedMetricAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent,
Builder subfactoriesBuilder) throws IOException {
context.searchContext().markAsNotCachable();
return new ScriptedMetricAggregatorFactory(name, type, initScript, mapScript, combineScript, reduceScript, params, context,
QueryShardContext queryShardContext = context.searchContext().getQueryShardContext();
Function<Map<String, Object>, ExecutableScript> executableInitScript;
if (initScript != null) {
executableInitScript = queryShardContext.getLazyExecutableScript(initScript, ScriptContext.Standard.AGGS,
Collections.emptyMap());
} else {
executableInitScript = (p) -> null;;
}
Function<Map<String, Object>, SearchScript> searchMapScript = queryShardContext.getLazySearchScript(mapScript,
ScriptContext.Standard.AGGS, Collections.emptyMap());
Function<Map<String, Object>, ExecutableScript> executableCombineScript;
if (combineScript != null) {
executableCombineScript = queryShardContext.getLazyExecutableScript(combineScript, ScriptContext.Standard.AGGS,
Collections.emptyMap());
} else {
executableCombineScript = (p) -> null;
}
return new ScriptedMetricAggregatorFactory(name, type, searchMapScript, executableInitScript, executableCombineScript, reduceScript, params, context,
parent, subfactoriesBuilder, metaData);
}
@Override
protected XContentBuilder internalXContent(XContentBuilder builder, Params builderParams) throws IOException {
builder.startObject();

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.aggregations.metrics.scripted;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.LeafSearchScript;
import org.elasticsearch.script.Script;
@ -46,23 +47,14 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
private final Script reduceScript;
private Map<String, Object> params;
protected ScriptedMetricAggregator(String name, Script initScript, Script mapScript, Script combineScript, Script reduceScript,
protected ScriptedMetricAggregator(String name, SearchScript mapScript, ExecutableScript combineScript,
Script reduceScript,
Map<String, Object> params, AggregationContext context, Aggregator parent, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
throws IOException {
super(name, context, parent, pipelineAggregators, metaData);
this.params = params;
ScriptService scriptService = context.searchContext().scriptService();
if (initScript != null) {
context.searchContext().markAsNotCachable();
scriptService.executable(initScript, ScriptContext.Standard.AGGS, Collections.emptyMap()).run();
}
this.mapScript = scriptService.search(context.searchContext().lookup(), mapScript, ScriptContext.Standard.AGGS, Collections.emptyMap());
if (combineScript != null) {
context.searchContext().markAsNotCachable();
this.combineScript = scriptService.executable(combineScript, ScriptContext.Standard.AGGS, Collections.emptyMap());
} else {
this.combineScript = null;
}
this.mapScript = mapScript;
this.combineScript = combineScript;
this.reduceScript = reduceScript;
}

View File

@ -19,7 +19,11 @@
package org.elasticsearch.search.aggregations.metrics.scripted;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.script.CompiledScript;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -35,21 +39,22 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.function.Function;
public class ScriptedMetricAggregatorFactory extends AggregatorFactory<ScriptedMetricAggregatorFactory> {
private final Script initScript;
private final Script mapScript;
private final Script combineScript;
private final Function<Map<String, Object>, SearchScript> mapScript;
private final Function<Map<String, Object>, ExecutableScript> combineScript;
private final Script reduceScript;
private final Map<String, Object> params;
private final Function<Map<String, Object>, ExecutableScript> initScript;
public ScriptedMetricAggregatorFactory(String name, Type type, Script initScript, Script mapScript, Script combineScript,
Script reduceScript, Map<String, Object> params, AggregationContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
public ScriptedMetricAggregatorFactory(String name, Type type, Function<Map<String, Object>, SearchScript> mapScript, Function<Map<String, Object>, ExecutableScript> initScript, Function<Map<String, Object>, ExecutableScript> combineScript,
Script reduceScript, Map<String, Object> params, AggregationContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
this.initScript = initScript;
this.mapScript = mapScript;
this.initScript = initScript;
this.combineScript = combineScript;
this.reduceScript = reduceScript;
this.params = params;
@ -68,16 +73,18 @@ public class ScriptedMetricAggregatorFactory extends AggregatorFactory<ScriptedM
params = new HashMap<>();
params.put("_agg", new HashMap<String, Object>());
}
return new ScriptedMetricAggregator(name, insertParams(initScript, params), insertParams(mapScript, params),
insertParams(combineScript, params), deepCopyScript(reduceScript, context.searchContext()), params, context, parent,
pipelineAggregators, metaData);
}
private static Script insertParams(Script script, Map<String, Object> params) {
if (script == null) {
return null;
final ExecutableScript initScript = this.initScript.apply(params);
final SearchScript mapScript = this.mapScript.apply(params);
final ExecutableScript combineScript = this.combineScript.apply(params);
final Script reduceScript = deepCopyScript(this.reduceScript, context.searchContext());
if (initScript != null) {
initScript.run();
}
return new Script(script.getScript(), script.getType(), script.getLang(), params);
return new ScriptedMetricAggregator(name, mapScript,
combineScript, reduceScript, params, context, parent,
pipelineAggregators, metaData);
}
private static Script deepCopyScript(Script script, SearchContext context) {
@ -98,7 +105,7 @@ public class ScriptedMetricAggregatorFactory extends AggregatorFactory<ScriptedM
if (original instanceof Map) {
Map<?, ?> originalMap = (Map<?, ?>) original;
Map<Object, Object> clonedMap = new HashMap<>();
for (Entry<?, ?> e : originalMap.entrySet()) {
for (Map.Entry<?, ?> e : originalMap.entrySet()) {
clonedMap.put(deepCopyParams(e.getKey(), context), deepCopyParams(e.getValue(), context));
}
clone = (T) clonedMap;
@ -110,14 +117,15 @@ public class ScriptedMetricAggregatorFactory extends AggregatorFactory<ScriptedM
}
clone = (T) clonedList;
} else if (original instanceof String || original instanceof Integer || original instanceof Long || original instanceof Short
|| original instanceof Byte || original instanceof Float || original instanceof Double || original instanceof Character
|| original instanceof Boolean) {
|| original instanceof Byte || original instanceof Float || original instanceof Double || original instanceof Character
|| original instanceof Boolean) {
clone = original;
} else {
throw new SearchParseException(context,
"Can only clone primitives, String, ArrayList, and HashMap. Found: " + original.getClass().getCanonicalName(), null);
"Can only clone primitives, String, ArrayList, and HashMap. Found: " + original.getClass().getCanonicalName(), null);
}
return clone;
}
}

View File

@ -28,6 +28,8 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationInitializationException;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@ -39,6 +41,7 @@ import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
import org.elasticsearch.search.fetch.StoredFieldsContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.ScriptFieldsContext;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.sort.ScoreSortBuilder;
import org.elasticsearch.search.sort.SortBuilder;
@ -527,11 +530,17 @@ public class TopHitsAggregationBuilder extends AbstractAggregationBuilder<TopHit
@Override
protected TopHitsAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subfactoriesBuilder)
throws IOException {
if (scriptFields != null && scriptFields.isEmpty() == false) {
context.searchContext().markAsNotCachable();
List<ScriptFieldsContext.ScriptField> fields = new ArrayList<>();
if (scriptFields != null) {
for (ScriptField field : scriptFields) {
SearchScript searchScript = context.searchContext().getQueryShardContext().getSearchScript(field.script(),
ScriptContext.Standard.SEARCH, Collections.emptyMap());
fields.add(new org.elasticsearch.search.fetch.subphase.ScriptFieldsContext.ScriptField(
field.fieldName(), searchScript, field.ignoreFailure()));
}
}
return new TopHitsAggregatorFactory(name, type, from, size, explain, version, trackScores, sorts, highlightBuilder,
storedFieldsContext, fieldDataFields, scriptFields, fetchSourceContext, context,
storedFieldsContext, fieldDataFields, fields, fetchSourceContext, context,
parent, subfactoriesBuilder, metaData);
}

View File

@ -31,6 +31,7 @@ import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField;
import org.elasticsearch.search.fetch.StoredFieldsContext;
import org.elasticsearch.search.fetch.subphase.DocValueFieldsContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.ScriptFieldsContext;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.internal.SubSearchContext;
import org.elasticsearch.search.sort.SortAndFormats;
@ -54,14 +55,14 @@ public class TopHitsAggregatorFactory extends AggregatorFactory<TopHitsAggregato
private final HighlightBuilder highlightBuilder;
private final StoredFieldsContext storedFieldsContext;
private final List<String> docValueFields;
private final Set<ScriptField> scriptFields;
private final List<ScriptFieldsContext.ScriptField> scriptFields;
private final FetchSourceContext fetchSourceContext;
public TopHitsAggregatorFactory(String name, Type type, int from, int size, boolean explain, boolean version, boolean trackScores,
List<SortBuilder<?>> sorts, HighlightBuilder highlightBuilder, StoredFieldsContext storedFieldsContext,
List<String> docValueFields, Set<ScriptField> scriptFields, FetchSourceContext fetchSourceContext,
AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
List<SortBuilder<?>> sorts, HighlightBuilder highlightBuilder, StoredFieldsContext storedFieldsContext,
List<String> docValueFields, List<ScriptFieldsContext.ScriptField> scriptFields, FetchSourceContext fetchSourceContext,
AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
this.from = from;
this.size = size;
@ -99,12 +100,8 @@ public class TopHitsAggregatorFactory extends AggregatorFactory<TopHitsAggregato
subSearchContext.docValueFieldsContext(new DocValueFieldsContext(docValueFields));
}
if (scriptFields != null) {
subSearchContext.markAsNotCachable();
for (ScriptField field : scriptFields) {
SearchScript searchScript = subSearchContext.scriptService().search(subSearchContext.lookup(), field.script(),
ScriptContext.Standard.SEARCH, Collections.emptyMap());
subSearchContext.scriptFields().add(new org.elasticsearch.search.fetch.subphase.ScriptFieldsContext.ScriptField(
field.fieldName(), searchScript, field.ignoreFailure()));
for (ScriptFieldsContext.ScriptField field : scriptFields) {
subSearchContext.scriptFields().add(field);
}
}
if (fetchSourceContext != null) {

View File

@ -97,7 +97,7 @@ public class AggregationContext {
} else {
if (config.fieldContext() != null && config.fieldContext().fieldType() != null) {
missing = config.fieldContext().fieldType().docValueFormat(null, DateTimeZone.UTC)
.parseDouble(config.missing().toString(), false, context::nowInMillis);
.parseDouble(config.missing().toString(), false, context.getQueryShardContext()::nowInMillis);
} else {
missing = Double.parseDouble(config.missing().toString());
}

View File

@ -379,8 +379,7 @@ public abstract class ValuesSourceAggregationBuilder<VS extends ValuesSource, AB
if (script == null) {
return null;
} else {
context.markAsNotCachable();
return context.scriptService().search(context.lookup(), script, ScriptContext.Standard.AGGS, Collections.emptyMap());
return context.getQueryShardContext().getSearchScript(script, ScriptContext.Standard.AGGS, Collections.emptyMap());
}
}

View File

@ -153,11 +153,6 @@ public abstract class FilteredSearchContext extends SearchContext {
return in.getOriginNanoTime();
}
@Override
protected long nowInMillisImpl() {
return in.nowInMillisImpl();
}
@Override
public ScrollContext scrollContext() {
return in.scrollContext();
@ -263,11 +258,6 @@ public abstract class FilteredSearchContext extends SearchContext {
return in.similarityService();
}
@Override
public ScriptService scriptService() {
return in.scriptService();
}
@Override
public BigArrays bigArrays() {
return in.bigArrays();

View File

@ -118,8 +118,6 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
}
}
private boolean canCache = true;
@Override
protected final void closeInternal() {
try {
@ -162,21 +160,14 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
public abstract long getOriginNanoTime();
public final long nowInMillis() {
markAsNotCachable();
return nowInMillisImpl();
}
public final boolean isCachable() {
return canCache;
return getQueryShardContext().isCachable();
}
public final void resetCanCache() {
this.canCache = true;
getQueryShardContext().setCachabe(true);
}
protected abstract long nowInMillisImpl();
public abstract ScrollContext scrollContext();
public abstract SearchContext scrollContext(ScrollContext scroll);
@ -238,8 +229,6 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
public abstract SimilarityService similarityService();
public abstract ScriptService scriptService();
public abstract BigArrays bigArrays();
public abstract BitsetFilterCache bitsetFilterCache();
@ -402,10 +391,6 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
/** Return a view of the additional query collectors that should be run for this context. */
public abstract Map<Class<?>, Collector> queryCollectors();
public final void markAsNotCachable() {
this.canCache = false;
}
/**
* The life time of an object that is used during search execution.
*/

View File

@ -284,9 +284,7 @@ public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
@Override
public SortFieldAndFormat build(QueryShardContext context) throws IOException {
context.markAsNotCachable();
final SearchScript searchScript = context.getScriptService().search(
context.lookup(), script, ScriptContext.Standard.SEARCH, Collections.emptyMap());
final SearchScript searchScript = context.getSearchScript(script, ScriptContext.Standard.SEARCH, Collections.emptyMap());
MultiValueMode valueMode = null;
if (sortMode != null) {

View File

@ -55,6 +55,7 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
private final BytesRef SEPARATOR = new BytesRef(" ");
@ -109,7 +110,7 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
response.addTerm(resultEntry);
final BytesRefBuilder byteSpare = new BytesRefBuilder();
final CompiledScript collateScript = suggestion.getCollateQueryScript();
final Function<Map<String, Object>, ExecutableScript> collateScript = suggestion.getCollateQueryScript();
final boolean collatePrune = (collateScript != null) && suggestion.collatePrune();
for (int i = 0; i < checkerResult.corrections.length; i++) {
Correction correction = checkerResult.corrections[i];
@ -121,8 +122,7 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
final Map<String, Object> vars = suggestion.getCollateScriptParams();
vars.put(SUGGESTION_TEMPLATE_VAR_NAME, spare.toString());
QueryShardContext shardContext = suggestion.getShardContext();
ScriptService scriptService = shardContext.getScriptService();
final ExecutableScript executable = scriptService.executable(collateScript, vars);
final ExecutableScript executable = collateScript.apply(vars);
final BytesReference querySource = (BytesReference) executable.run();
try (XContentParser parser = XContentFactory.xContent(querySource).createParser(querySource)) {
Optional<QueryBuilder> innerQueryBuilder = shardContext.newParseContext(parser).parseInnerQueryBuilder();

View File

@ -40,6 +40,7 @@ import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
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;
@ -56,6 +57,7 @@ import java.util.Map;
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 ( <tt>phrase</tt>).
@ -633,9 +635,8 @@ public class PhraseSuggestionBuilder extends SuggestionBuilder<PhraseSuggestionB
}
if (this.collateQuery != null) {
context.markAsNotCachable();
CompiledScript compiledScript = context.getScriptService().compile(this.collateQuery, ScriptContext.Standard.SEARCH,
Collections.emptyMap());
Function<Map<String, Object>, ExecutableScript> compiledScript = context.getLazyExecutableScript(this.collateQuery,
ScriptContext.Standard.SEARCH, Collections.emptyMap());
suggestionContext.setCollateQueryScript(compiledScript);
if (this.collateParams != null) {
suggestionContext.setCollateScriptParams(this.collateParams);

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.util.BytesRef;
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;
@ -31,6 +32,7 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
class PhraseSuggestionContext extends SuggestionContext {
static final boolean DEFAULT_COLLATE_PRUNE = false;
@ -52,7 +54,7 @@ class PhraseSuggestionContext extends SuggestionContext {
private boolean requireUnigram = DEFAULT_REQUIRE_UNIGRAM;
private BytesRef preTag;
private BytesRef postTag;
private CompiledScript collateQueryScript;
private Function<Map<String, Object>, ExecutableScript> collateQueryScript;
private boolean prune = DEFAULT_COLLATE_PRUNE;
private List<DirectCandidateGenerator> generators = new ArrayList<>();
private Map<String, Object> collateScriptParams = new HashMap<>(1);
@ -192,11 +194,11 @@ class PhraseSuggestionContext extends SuggestionContext {
return postTag;
}
CompiledScript getCollateQueryScript() {
Function<Map<String, Object>, ExecutableScript> getCollateQueryScript() {
return collateQueryScript;
}
void setCollateQueryScript(CompiledScript collateQueryScript) {
void setCollateQueryScript( Function<Map<String, Object>, ExecutableScript> collateQueryScript) {
this.collateQueryScript = collateQueryScript;
}

View File

@ -46,8 +46,7 @@ public class SearchSlowLogTests extends ESSingleNodeTestCase {
protected SearchContext createSearchContext(IndexService indexService) {
BigArrays bigArrays = indexService.getBigArrays();
ThreadPool threadPool = indexService.getThreadPool();
ScriptService scriptService = node().injector().getInstance(ScriptService.class);
return new TestSearchContext(threadPool, bigArrays, scriptService, indexService) {
return new TestSearchContext(threadPool, bigArrays, indexService) {
@Override
public ShardSearchRequest request() {
return new ShardSearchRequest() {

View File

@ -47,8 +47,8 @@ public class QueryShardContextTests extends ESTestCase {
MapperService mapperService = mock(MapperService.class);
when(mapperService.getIndexSettings()).thenReturn(indexSettings);
QueryShardContext context = new QueryShardContext(
indexSettings, null, null, mapperService, null, null, null, null, null, null
);
indexSettings, null, null, mapperService, null, null, null, null, null, null,
System::currentTimeMillis);
context.setAllowUnmappedFields(false);
MappedFieldType fieldType = new TextFieldMapper.TextFieldType();

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.bucket.histogram.ExtendedBounds;
@ -93,7 +94,9 @@ public class ExtendedBoundsTests extends ESTestCase {
public void testParseAndValidate() {
long now = randomLong();
SearchContext context = mock(SearchContext.class);
when(context.nowInMillis()).thenReturn(now);
QueryShardContext qsc = mock(QueryShardContext.class);
when(context.getQueryShardContext()).thenReturn(qsc);
when(qsc.nowInMillis()).thenReturn(now);
FormatDateTimeFormatter formatter = Joda.forPattern("dateOptionalTime");
DocValueFormat format = new DocValueFormat.DateTime(formatter, DateTimeZone.UTC);

View File

@ -49,9 +49,6 @@ import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.TermQueryBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.fetch.subphase.highlight.AbstractHighlighterBuilder;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder.Field;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder.Order;
import org.elasticsearch.search.fetch.subphase.highlight.SearchContextHighlight.FieldOptions;
@ -294,7 +291,7 @@ public class HighlightBuilderTests extends ESTestCase {
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings);
// shard context will only need indicesQueriesRegistry for building Query objects nested in highlighter
QueryShardContext mockShardContext = new QueryShardContext(idxSettings, null, null, null, null, null, indicesQueriesRegistry,
null, null, null) {
null, null, null, System::currentTimeMillis) {
@Override
public MappedFieldType fieldMapper(String name) {
TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name);

View File

@ -161,7 +161,7 @@ public class QueryRescoreBuilderTests extends ESTestCase {
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(idxSettings, null, null, null, null, null, indicesQueriesRegistry,
null, null, null) {
null, null, null, System::currentTimeMillis) {
@Override
public MappedFieldType fieldMapper(String name) {
TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name);

View File

@ -24,7 +24,6 @@ import org.apache.lucene.util.Accountable;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
@ -236,7 +235,7 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
}
});
return new QueryShardContext(idxSettings, bitsetFilterCache, ifds, null, null, scriptService,
indicesQueriesRegistry, null, null, null) {
indicesQueriesRegistry, null, null, null, System::currentTimeMillis) {
@Override
public MappedFieldType fieldMapper(String name) {
return provideMappedFieldType(name);

View File

@ -120,10 +120,7 @@ public class TemplateQueryBuilder extends AbstractQueryBuilder<TemplateQueryBuil
@Override
protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws IOException {
queryRewriteContext.markAsNotCachable();
ExecutableScript executable = queryRewriteContext.getScriptService().executable(template,
ScriptContext.Standard.SEARCH, Collections.emptyMap());
BytesReference querySource = (BytesReference) executable.run();
BytesReference querySource = queryRewriteContext.getTemplateBytes(template);
try (XContentParser qSourceParser = XContentFactory.xContent(querySource).createParser(querySource)) {
final QueryParseContext queryParseContext = queryRewriteContext.newParseContext(qSourceParser);
final QueryBuilder queryBuilder = queryParseContext.parseInnerQueryBuilder().orElseThrow(

View File

@ -1110,7 +1110,7 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
QueryShardContext createShardContext() {
ClusterState state = ClusterState.builder(new ClusterName("_name")).build();
return new QueryShardContext(idxSettings, bitsetFilterCache, indexFieldDataService, mapperService, similarityService,
scriptService, indicesQueriesRegistry, this.client, null, state);
scriptService, indicesQueriesRegistry, this.client, null, state, System::currentTimeMillis);
}
ScriptModule createScriptModule(List<ScriptPlugin> scriptPlugins) {

View File

@ -283,7 +283,7 @@ public abstract class ESSingleNodeTestCase extends ESTestCase {
BigArrays bigArrays = indexService.getBigArrays();
ThreadPool threadPool = indexService.getThreadPool();
ScriptService scriptService = node().injector().getInstance(ScriptService.class);
return new TestSearchContext(threadPool, bigArrays, scriptService, indexService);
return new TestSearchContext(threadPool, bigArrays, indexService);
}
/**

View File

@ -37,7 +37,6 @@ import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchExtBuilder;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.SearchContextAggregations;
@ -77,7 +76,6 @@ public class TestSearchContext extends SearchContext {
final Counter timeEstimateCounter = Counter.newCounter();
final QuerySearchResult queryResult = new QuerySearchResult();
final QueryShardContext queryShardContext;
ScriptService scriptService;
ParsedQuery originalQuery;
ParsedQuery postFilter;
Query query;
@ -91,7 +89,7 @@ public class TestSearchContext extends SearchContext {
private final long originNanoTime = System.nanoTime();
private final Map<String, SearchExtBuilder> searchExtBuilders = new HashMap<>();
public TestSearchContext(ThreadPool threadPool, BigArrays bigArrays, ScriptService scriptService, IndexService indexService) {
public TestSearchContext(ThreadPool threadPool, BigArrays bigArrays, IndexService indexService) {
super(ParseFieldMatcher.STRICT);
this.bigArrays = bigArrays.withCircuitBreaking();
this.indexService = indexService;
@ -99,7 +97,6 @@ public class TestSearchContext extends SearchContext {
this.fixedBitSetFilterCache = indexService.cache().bitsetFilterCache();
this.threadPool = threadPool;
this.indexShard = indexService.getShardOrNull(0);
this.scriptService = scriptService;
queryShardContext = indexService.newQueryShardContext();
}
@ -111,7 +108,6 @@ public class TestSearchContext extends SearchContext {
this.threadPool = null;
this.fixedBitSetFilterCache = null;
this.indexShard = null;
scriptService = null;
this.queryShardContext = queryShardContext;
}
@ -169,11 +165,6 @@ public class TestSearchContext extends SearchContext {
return originNanoTime;
}
@Override
protected long nowInMillisImpl() {
return 0;
}
@Override
public ScrollContext scrollContext() {
return null;
@ -299,11 +290,6 @@ public class TestSearchContext extends SearchContext {
return null;
}
@Override
public ScriptService scriptService() {
return scriptService;
}
@Override
public BigArrays bigArrays() {
return bigArrays;

View File

@ -34,7 +34,7 @@ import org.elasticsearch.test.TestSearchContext;
public class MockSearchServiceTests extends ESTestCase {
public void testAssertNoInFlightContext() {
SearchContext s = new TestSearchContext(new QueryShardContext(new IndexSettings(IndexMetaData.PROTO, Settings.EMPTY), null, null,
null, null, null, null, null, null, null)) {
null, null, null, null, null, null, null, System::currentTimeMillis)) {
@Override
public SearchShardTarget shardTarget() {
return new SearchShardTarget("node", new Index("idx", "ignored"), 0);