#20750 Prevent requests that use scripts or now() from being cached

Prevent requests that use scripts or now() from being cached
This commit is contained in:
Colin Goodheart-Smithe 2016-10-06 10:24:57 +01:00 committed by GitHub
commit 3aed047bde
75 changed files with 1385 additions and 325 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -19,17 +19,24 @@
package org.elasticsearch.index.query; package org.elasticsearch.index.query;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParseFieldMatcherSupplier; import org.elasticsearch.common.ParseFieldMatcherSupplier;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.indices.query.IndicesQueriesRegistry; 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.ScriptService;
import org.elasticsearch.script.ScriptSettings; import org.elasticsearch.script.ScriptSettings;
import java.util.Collections;
/** /**
* Context object used to rewrite {@link QueryBuilder} instances into simplified version. * Context object used to rewrite {@link QueryBuilder} instances into simplified version.
*/ */
@ -69,13 +76,6 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier {
return indexSettings; return indexSettings;
} }
/**
* Returns a script service to fetch scripts.
*/
public final ScriptService getScriptService() {
return scriptService;
}
/** /**
* Return the MapperService. * Return the MapperService.
*/ */
@ -116,4 +116,12 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier {
String defaultScriptLanguage = ScriptSettings.getLegacyDefaultLang(indexSettings.getNodeSettings()); String defaultScriptLanguage = ScriptSettings.getLegacyDefaultLang(indexSettings.getNodeSettings());
return new QueryParseContext(defaultScriptLanguage, indicesQueriesRegistry, parser, indexSettings.getParseFieldMatcher()); return new QueryParseContext(defaultScriptLanguage, indicesQueriesRegistry, parser, indexSettings.getParseFieldMatcher());
} }
public BytesReference getTemplateBytes(Script template) {
ExecutableScript executable = scriptService.executable(template,
ScriptContext.Standard.SEARCH, Collections.emptyMap());
return (BytesReference) executable.run();
}
} }

View File

@ -26,6 +26,9 @@ import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.function.Function;
import java.util.function.LongSupplier;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queryparser.classic.MapperQueryParser; import org.apache.lucene.queryparser.classic.MapperQueryParser;
@ -33,11 +36,13 @@ import org.apache.lucene.queryparser.classic.QueryParserSettings;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.search.join.BitSetProducer; import org.apache.lucene.search.join.BitSetProducer;
import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
@ -54,7 +59,12 @@ import org.elasticsearch.index.mapper.TextFieldMapper;
import org.elasticsearch.index.query.support.NestedScope; import org.elasticsearch.index.query.support.NestedScope;
import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.indices.query.IndicesQueriesRegistry; 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.ScriptService;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.lookup.SearchLookup;
@ -69,6 +79,8 @@ public class QueryShardContext extends QueryRewriteContext {
private final IndexFieldDataService indexFieldDataService; private final IndexFieldDataService indexFieldDataService;
private final IndexSettings indexSettings; private final IndexSettings indexSettings;
private String[] types = Strings.EMPTY_ARRAY; private String[] types = Strings.EMPTY_ARRAY;
private boolean cachable = true;
private final SetOnce<Boolean> frozen = new SetOnce<>();
public void setTypes(String... types) { public void setTypes(String... types) {
this.types = types; this.types = types;
@ -85,11 +97,12 @@ public class QueryShardContext extends QueryRewriteContext {
private boolean mapUnmappedFieldAsString; private boolean mapUnmappedFieldAsString;
private NestedScope nestedScope; private NestedScope nestedScope;
private boolean isFilter; private boolean isFilter;
private final LongSupplier nowInMillis;
public QueryShardContext(IndexSettings indexSettings, BitsetFilterCache bitsetFilterCache, IndexFieldDataService indexFieldDataService, public QueryShardContext(IndexSettings indexSettings, BitsetFilterCache bitsetFilterCache, IndexFieldDataService indexFieldDataService,
MapperService mapperService, SimilarityService similarityService, ScriptService scriptService, MapperService mapperService, SimilarityService similarityService, ScriptService scriptService,
final IndicesQueriesRegistry indicesQueriesRegistry, Client client, final IndicesQueriesRegistry indicesQueriesRegistry, Client client,
IndexReader reader, ClusterState clusterState) { IndexReader reader, ClusterState clusterState, LongSupplier nowInMillis) {
super(indexSettings, mapperService, scriptService, indicesQueriesRegistry, client, reader, clusterState); super(indexSettings, mapperService, scriptService, indicesQueriesRegistry, client, reader, clusterState);
this.indexSettings = indexSettings; this.indexSettings = indexSettings;
this.similarityService = similarityService; this.similarityService = similarityService;
@ -99,12 +112,13 @@ public class QueryShardContext extends QueryRewriteContext {
this.allowUnmappedFields = indexSettings.isDefaultAllowUnmappedFields(); this.allowUnmappedFields = indexSettings.isDefaultAllowUnmappedFields();
this.indicesQueriesRegistry = indicesQueriesRegistry; this.indicesQueriesRegistry = indicesQueriesRegistry;
this.nestedScope = new NestedScope(); this.nestedScope = new NestedScope();
this.nowInMillis = nowInMillis;
} }
public QueryShardContext(QueryShardContext source) { public QueryShardContext(QueryShardContext source) {
this(source.indexSettings, source.bitsetFilterCache, source.indexFieldDataService, source.mapperService, this(source.indexSettings, source.bitsetFilterCache, source.indexFieldDataService, source.mapperService,
source.similarityService, source.scriptService, source.indicesQueriesRegistry, source.client, source.similarityService, source.scriptService, source.indicesQueriesRegistry, source.client,
source.reader, source.clusterState); source.reader, source.clusterState, source.nowInMillis);
this.types = source.getTypes(); this.types = source.getTypes();
} }
@ -261,11 +275,8 @@ public class QueryShardContext extends QueryRewriteContext {
} }
public long nowInMillis() { public long nowInMillis() {
SearchContext current = SearchContext.current(); failIfFrozen();
if (current != null) { return nowInMillis.getAsLong();
return current.nowInMillis();
}
return System.currentTimeMillis();
} }
public NestedScope nestedScope() { public NestedScope nestedScope() {
@ -327,4 +338,77 @@ public class QueryShardContext extends QueryRewriteContext {
public final Index index() { public final Index index() {
return indexSettings.getIndex(); return indexSettings.getIndex();
} }
/**
* Compiles (or retrieves from cache) and binds the parameters to the
* provided script
*/
public SearchScript getSearchScript(Script script, ScriptContext context, Map<String, String> params) {
failIfFrozen();
return scriptService.search(lookup(), script, context, params);
}
/**
* Returns a lazily created {@link SearchScript} that is compiled immediately but can be pulled later once all
* parameters are available.
*/
public Function<Map<String, Object>, SearchScript> getLazySearchScript(Script script, ScriptContext context,
Map<String, String> params) {
failIfFrozen();
CompiledScript compile = scriptService.compile(script, context, params);
return (p) -> scriptService.search(lookup(), compile, p);
}
/**
* Compiles (or retrieves from cache) and binds the parameters to the
* provided script
*/
public ExecutableScript getExecutableScript(Script script, ScriptContext context, Map<String, String> params) {
failIfFrozen();
return scriptService.executable(script, context, params);
}
/**
* Returns a lazily created {@link ExecutableScript} that is compiled immediately but can be pulled later once all
* parameters are available.
*/
public Function<Map<String, Object>, ExecutableScript> getLazyExecutableScript(Script script, ScriptContext context,
Map<String, String> params) {
failIfFrozen();
CompiledScript executable = scriptService.compile(script, context, params);
return (p) -> scriptService.executable(executable, p);
}
/**
* if this method is called the query context will throw exception if methods are accessed
* that could yield different results across executions like {@link #getTemplateBytes(Script)}
*/
public void freezeContext() {
this.frozen.set(Boolean.TRUE);
}
/**
* This method fails if {@link #freezeContext()} is called before on this context.
* This is used to <i>seal</i>
*/
protected void failIfFrozen() {
this.cachable = false;
if (frozen.get() == Boolean.TRUE) {
throw new IllegalArgumentException("features that prevent cachability are disabled on this context");
} else {
assert frozen.get() == null : frozen.get();
}
}
@Override
public BytesReference getTemplateBytes(Script template) {
failIfFrozen();
return super.getTemplateBytes(template);
}
/**
* Returns <code>true</code> iff the result of the processed search request is cachable. Otherwise <code>false</code>
*/
public boolean isCachable() {
return cachable;
}
} }

View File

@ -41,7 +41,6 @@ import org.elasticsearch.search.lookup.SearchLookup;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.Optional; import java.util.Optional;
@ -134,7 +133,7 @@ public class ScriptQueryBuilder extends AbstractQueryBuilder<ScriptQueryBuilder>
@Override @Override
protected Query doToQuery(QueryShardContext context) throws IOException { protected Query doToQuery(QueryShardContext context) throws IOException {
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 { static class ScriptQuery extends Query {
@ -143,9 +142,9 @@ public class ScriptQueryBuilder extends AbstractQueryBuilder<ScriptQueryBuilder>
private final SearchScript searchScript; private final SearchScript searchScript;
public ScriptQuery(Script script, ScriptService scriptService, SearchLookup searchLookup) { public ScriptQuery(Script script, SearchScript searchScript) {
this.script = script; this.script = script;
this.searchScript = scriptService.search(searchLookup, script, ScriptContext.Standard.SEARCH, Collections.emptyMap()); this.searchScript = searchScript;
} }
@Override @Override
@ -216,4 +215,6 @@ public class ScriptQueryBuilder extends AbstractQueryBuilder<ScriptQueryBuilder>
protected boolean doEquals(ScriptQueryBuilder other) { protected boolean doEquals(ScriptQueryBuilder other) {
return Objects.equals(script, other.script); return Objects.equals(script, other.script);
} }
} }

View File

@ -49,12 +49,19 @@ public class RandomScoreFunctionBuilder extends ScoreFunctionBuilder<RandomScore
*/ */
public RandomScoreFunctionBuilder(StreamInput in) throws IOException { public RandomScoreFunctionBuilder(StreamInput in) throws IOException {
super(in); super(in);
if (in.readBoolean()) {
seed = in.readInt(); seed = in.readInt();
} }
}
@Override @Override
protected void doWriteTo(StreamOutput out) throws IOException { protected void doWriteTo(StreamOutput out) throws IOException {
if (seed != null) {
out.writeBoolean(true);
out.writeInt(seed); out.writeInt(seed);
} else {
out.writeBoolean(false);
}
} }
@Override @Override

View File

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

View File

@ -1082,7 +1082,7 @@ public class IndicesService extends AbstractLifecycleComponent
} }
// if now in millis is used (or in the future, a more generic "isDeterministic" flag // if now in millis is used (or in the future, a more generic "isDeterministic" flag
// then we can't cache based on "now" key within the search request, as it is not deterministic // then we can't cache based on "now" key within the search request, as it is not deterministic
if (context.nowInMillisUsed()) { if (context.getQueryShardContext().isCachable() == false) {
return false; return false;
} }
return true; return true;

View File

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

View File

@ -249,7 +249,7 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
long timePassed = now - lastInlineCompileTime; long timePassed = now - lastInlineCompileTime;
lastInlineCompileTime = now; lastInlineCompileTime = now;
scriptsPerMinCounter += ((double) timePassed) * compilesAllowedPerNano; scriptsPerMinCounter += (timePassed) * compilesAllowedPerNano;
// It's been over the time limit anyway, readjust the bucket to be level // It's been over the time limit anyway, readjust the bucket to be level
if (scriptsPerMinCounter > totalCompilesPerMinute) { if (scriptsPerMinCounter > totalCompilesPerMinute) {
@ -488,7 +488,15 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
*/ */
public SearchScript search(SearchLookup lookup, Script script, ScriptContext scriptContext, Map<String, String> params) { public SearchScript search(SearchLookup lookup, Script script, ScriptContext scriptContext, Map<String, String> params) {
CompiledScript compiledScript = compile(script, scriptContext, params); CompiledScript compiledScript = compile(script, scriptContext, params);
return getScriptEngineServiceForLang(compiledScript.lang()).search(compiledScript, lookup, script.getParams()); return search(lookup, compiledScript, script.getParams());
}
/**
* Binds provided parameters to a compiled script returning a
* {@link SearchScript} ready for execution
*/
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) { 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.query.QueryShardContext;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.aggregations.SearchContextAggregations; import org.elasticsearch.search.aggregations.SearchContextAggregations;
import org.elasticsearch.search.dfs.DfsSearchResult; import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.fetch.FetchPhase; import org.elasticsearch.search.fetch.FetchPhase;
@ -89,7 +88,6 @@ final class DefaultSearchContext extends SearchContext {
private final Counter timeEstimateCounter; private final Counter timeEstimateCounter;
private SearchType searchType; private SearchType searchType;
private final Engine.Searcher engineSearcher; private final Engine.Searcher engineSearcher;
private final ScriptService scriptService;
private final BigArrays bigArrays; private final BigArrays bigArrays;
private final IndexShard indexShard; private final IndexShard indexShard;
private final IndexService indexService; private final IndexService indexService;
@ -150,7 +148,7 @@ final class DefaultSearchContext extends SearchContext {
private FetchPhase fetchPhase; private FetchPhase fetchPhase;
DefaultSearchContext(long id, ShardSearchRequest request, SearchShardTarget shardTarget, Engine.Searcher engineSearcher, DefaultSearchContext(long id, ShardSearchRequest request, SearchShardTarget shardTarget, Engine.Searcher engineSearcher,
IndexService indexService, IndexShard indexShard, ScriptService scriptService, IndexService indexService, IndexShard indexShard,
BigArrays bigArrays, Counter timeEstimateCounter, ParseFieldMatcher parseFieldMatcher, TimeValue timeout, BigArrays bigArrays, Counter timeEstimateCounter, ParseFieldMatcher parseFieldMatcher, TimeValue timeout,
FetchPhase fetchPhase) { FetchPhase fetchPhase) {
super(parseFieldMatcher); super(parseFieldMatcher);
@ -160,7 +158,6 @@ final class DefaultSearchContext extends SearchContext {
this.searchType = request.searchType(); this.searchType = request.searchType();
this.shardTarget = shardTarget; this.shardTarget = shardTarget;
this.engineSearcher = engineSearcher; this.engineSearcher = engineSearcher;
this.scriptService = scriptService;
// SearchContexts use a BigArrays that can circuit break // SearchContexts use a BigArrays that can circuit break
this.bigArrays = bigArrays.withCircuitBreaking(); this.bigArrays = bigArrays.withCircuitBreaking();
this.dfsResult = new DfsSearchResult(id, shardTarget); this.dfsResult = new DfsSearchResult(id, shardTarget);
@ -171,10 +168,17 @@ final class DefaultSearchContext extends SearchContext {
this.searcher = new ContextIndexSearcher(engineSearcher, indexService.cache().query(), indexShard.getQueryCachingPolicy()); this.searcher = new ContextIndexSearcher(engineSearcher, indexService.cache().query(), indexShard.getQueryCachingPolicy());
this.timeEstimateCounter = timeEstimateCounter; this.timeEstimateCounter = timeEstimateCounter;
this.timeout = timeout; this.timeout = timeout;
queryShardContext = indexService.newQueryShardContext(searcher.getIndexReader()); queryShardContext = indexService.newQueryShardContext(searcher.getIndexReader(), request::nowInMillis);
queryShardContext.setTypes(request.types()); queryShardContext.setTypes(request.types());
} }
DefaultSearchContext(DefaultSearchContext source) {
this(source.id(), source.request(), source.shardTarget(), source.engineSearcher, source.indexService, source.indexShard(),
source.bigArrays(), source.timeEstimateCounter(), source.parseFieldMatcher(), source.timeout(), source.fetchPhase());
}
@Override @Override
public void doClose() { public void doClose() {
// clear and scope phase we have // clear and scope phase we have
@ -358,11 +362,6 @@ final class DefaultSearchContext extends SearchContext {
return originNanoTime; return originNanoTime;
} }
@Override
protected long nowInMillisImpl() {
return request.nowInMillis();
}
@Override @Override
public ScrollContext scrollContext() { public ScrollContext scrollContext() {
return this.scrollContext; return this.scrollContext;
@ -501,11 +500,6 @@ final class DefaultSearchContext extends SearchContext {
return indexService.similarityService(); return indexService.similarityService();
} }
@Override
public ScriptService scriptService() {
return scriptService;
}
@Override @Override
public BigArrays bigArrays() { public BigArrays bigArrays() {
return 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 { private void loadOrExecuteQueryPhase(final ShardSearchRequest request, final SearchContext context) throws Exception {
final boolean canCache = indicesService.canCache(request, context); final boolean canCache = indicesService.canCache(request, context);
context.getQueryShardContext().freezeContext();
if (canCache) { if (canCache) {
indicesService.loadIntoContext(request, context, queryPhase); indicesService.loadIntoContext(request, context, queryPhase);
} else { } else {
@ -516,17 +517,18 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
} }
final SearchContext createContext(ShardSearchRequest request, @Nullable Engine.Searcher searcher) throws IOException { final SearchContext createContext(ShardSearchRequest request, @Nullable Engine.Searcher searcher) throws IOException {
final DefaultSearchContext context = createSearchContext(request, defaultSearchTimeout, searcher);
DefaultSearchContext context = createSearchContext(request, defaultSearchTimeout, searcher);
SearchContext.setCurrent(context);
try { try {
request.rewrite(context.getQueryShardContext()); // we clone the search context here just for rewriting otherwise we
// reset that we have used nowInMillis from the context since it may // might end up with incorrect state since we are using now() or script services
// have been rewritten so its no longer in the query and the request can // during rewrite and normalized / evaluate templates etc.
// be cached. If it is still present in the request (e.g. in a range // NOTE this context doesn't need to be closed - the outer context will
// aggregation) it will still be caught when the aggregation is // take care of this.
// evaluated. DefaultSearchContext rewriteContext = new DefaultSearchContext(context);
context.resetNowInMillisUsed(); SearchContext.setCurrent(rewriteContext);
request.rewrite(rewriteContext.getQueryShardContext());
SearchContext.setCurrent(context);
assert context.getQueryShardContext().isCachable();
if (request.scroll() != null) { if (request.scroll() != null) {
context.scrollContext(new ScrollContext()); context.scrollContext(new ScrollContext());
context.scrollContext().scroll = request.scroll(); context.scrollContext().scroll = request.scroll();
@ -568,7 +570,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
return new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, engineSearcher, return new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, engineSearcher,
indexService, indexService,
indexShard, scriptService, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher, indexShard, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher,
timeout, fetchPhase); timeout, fetchPhase);
} }
@ -735,7 +737,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
} }
if (source.scriptFields() != null) { if (source.scriptFields() != null) {
for (org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField field : source.scriptFields()) { for (org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField field : source.scriptFields()) {
SearchScript searchScript = context.scriptService().search(context.lookup(), field.script(), ScriptContext.Standard.SEARCH, SearchScript searchScript = scriptService.search(context.lookup(), field.script(), ScriptContext.Standard.SEARCH,
Collections.emptyMap()); Collections.emptyMap());
context.scriptFields().add(new ScriptField(field.fieldName(), searchScript, field.ignoreFailure())); context.scriptFields().add(new ScriptField(field.fieldName(), searchScript, field.ignoreFailure()));
} }

View File

@ -21,6 +21,8 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.rounding.DateTimeUnit;
import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
@ -35,8 +37,12 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceType; import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects; import java.util.Objects;
import static java.util.Collections.unmodifiableMap;
/** /**
* A builder for histograms on date fields. * A builder for histograms on date fields.
*/ */
@ -44,6 +50,29 @@ public class DateHistogramAggregationBuilder
extends ValuesSourceAggregationBuilder<ValuesSource.Numeric, DateHistogramAggregationBuilder> { extends ValuesSourceAggregationBuilder<ValuesSource.Numeric, DateHistogramAggregationBuilder> {
public static final String NAME = InternalDateHistogram.TYPE.name(); public static final String NAME = InternalDateHistogram.TYPE.name();
public static final Map<String, DateTimeUnit> DATE_FIELD_UNITS;
static {
Map<String, DateTimeUnit> dateFieldUnits = new HashMap<>();
dateFieldUnits.put("year", DateTimeUnit.YEAR_OF_CENTURY);
dateFieldUnits.put("1y", DateTimeUnit.YEAR_OF_CENTURY);
dateFieldUnits.put("quarter", DateTimeUnit.QUARTER);
dateFieldUnits.put("1q", DateTimeUnit.QUARTER);
dateFieldUnits.put("month", DateTimeUnit.MONTH_OF_YEAR);
dateFieldUnits.put("1M", DateTimeUnit.MONTH_OF_YEAR);
dateFieldUnits.put("week", DateTimeUnit.WEEK_OF_WEEKYEAR);
dateFieldUnits.put("1w", DateTimeUnit.WEEK_OF_WEEKYEAR);
dateFieldUnits.put("day", DateTimeUnit.DAY_OF_MONTH);
dateFieldUnits.put("1d", DateTimeUnit.DAY_OF_MONTH);
dateFieldUnits.put("hour", DateTimeUnit.HOUR_OF_DAY);
dateFieldUnits.put("1h", DateTimeUnit.HOUR_OF_DAY);
dateFieldUnits.put("minute", DateTimeUnit.MINUTES_OF_HOUR);
dateFieldUnits.put("1m", DateTimeUnit.MINUTES_OF_HOUR);
dateFieldUnits.put("second", DateTimeUnit.SECOND_OF_MINUTE);
dateFieldUnits.put("1s", DateTimeUnit.SECOND_OF_MINUTE);
DATE_FIELD_UNITS = unmodifiableMap(dateFieldUnits);
}
private long interval; private long interval;
private DateHistogramInterval dateHistogramInterval; private DateHistogramInterval dateHistogramInterval;
private long offset = 0; private long offset = 0;
@ -245,8 +274,36 @@ public class DateHistogramAggregationBuilder
@Override @Override
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config, protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(AggregationContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException { AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
Rounding rounding = createRounding();
ExtendedBounds roundedBounds = null;
if (this.extendedBounds != null) {
// parse any string bounds to longs and round
roundedBounds = this.extendedBounds.parseAndValidate(name, context.searchContext(), config.format()).round(rounding);
}
return new DateHistogramAggregatorFactory(name, type, config, interval, dateHistogramInterval, offset, order, keyed, minDocCount, return new DateHistogramAggregatorFactory(name, type, config, interval, dateHistogramInterval, offset, order, keyed, minDocCount,
extendedBounds, context, parent, subFactoriesBuilder, metaData); rounding, roundedBounds, context, parent, subFactoriesBuilder, metaData);
}
private Rounding createRounding() {
Rounding.Builder tzRoundingBuilder;
if (dateHistogramInterval != null) {
DateTimeUnit dateTimeUnit = DATE_FIELD_UNITS.get(dateHistogramInterval.toString());
if (dateTimeUnit != null) {
tzRoundingBuilder = Rounding.builder(dateTimeUnit);
} else {
// the interval is a time value?
tzRoundingBuilder = Rounding.builder(
TimeValue.parseTimeValue(dateHistogramInterval.toString(), null, getClass().getSimpleName() + ".interval"));
}
} else {
// the interval is an integer time value in millis?
tzRoundingBuilder = Rounding.builder(TimeValue.timeValueMillis(interval));
}
if (timeZone() != null) {
tzRoundingBuilder.timeZone(timeZone());
}
Rounding rounding = tzRoundingBuilder.build();
return rounding;
} }
@Override @Override

View File

@ -19,9 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.histogram; package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.common.rounding.DateTimeUnit;
import org.elasticsearch.common.rounding.Rounding; import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
@ -30,12 +28,9 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import static java.util.Collections.unmodifiableMap;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
@ -44,29 +39,6 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
public final class DateHistogramAggregatorFactory public final class DateHistogramAggregatorFactory
extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, DateHistogramAggregatorFactory> { extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, DateHistogramAggregatorFactory> {
public static final Map<String, DateTimeUnit> DATE_FIELD_UNITS;
static {
Map<String, DateTimeUnit> dateFieldUnits = new HashMap<>();
dateFieldUnits.put("year", DateTimeUnit.YEAR_OF_CENTURY);
dateFieldUnits.put("1y", DateTimeUnit.YEAR_OF_CENTURY);
dateFieldUnits.put("quarter", DateTimeUnit.QUARTER);
dateFieldUnits.put("1q", DateTimeUnit.QUARTER);
dateFieldUnits.put("month", DateTimeUnit.MONTH_OF_YEAR);
dateFieldUnits.put("1M", DateTimeUnit.MONTH_OF_YEAR);
dateFieldUnits.put("week", DateTimeUnit.WEEK_OF_WEEKYEAR);
dateFieldUnits.put("1w", DateTimeUnit.WEEK_OF_WEEKYEAR);
dateFieldUnits.put("day", DateTimeUnit.DAY_OF_MONTH);
dateFieldUnits.put("1d", DateTimeUnit.DAY_OF_MONTH);
dateFieldUnits.put("hour", DateTimeUnit.HOUR_OF_DAY);
dateFieldUnits.put("1h", DateTimeUnit.HOUR_OF_DAY);
dateFieldUnits.put("minute", DateTimeUnit.MINUTES_OF_HOUR);
dateFieldUnits.put("1m", DateTimeUnit.MINUTES_OF_HOUR);
dateFieldUnits.put("second", DateTimeUnit.SECOND_OF_MINUTE);
dateFieldUnits.put("1s", DateTimeUnit.SECOND_OF_MINUTE);
DATE_FIELD_UNITS = unmodifiableMap(dateFieldUnits);
}
private final DateHistogramInterval dateHistogramInterval; private final DateHistogramInterval dateHistogramInterval;
private final long interval; private final long interval;
private final long offset; private final long offset;
@ -74,10 +46,11 @@ public final class DateHistogramAggregatorFactory
private final boolean keyed; private final boolean keyed;
private final long minDocCount; private final long minDocCount;
private final ExtendedBounds extendedBounds; private final ExtendedBounds extendedBounds;
private Rounding rounding;
public DateHistogramAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, long interval, public DateHistogramAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, long interval,
DateHistogramInterval dateHistogramInterval, long offset, InternalOrder order, boolean keyed, long minDocCount, DateHistogramInterval dateHistogramInterval, long offset, InternalOrder order, boolean keyed, long minDocCount,
ExtendedBounds extendedBounds, AggregationContext context, AggregatorFactory<?> parent, Rounding rounding, ExtendedBounds extendedBounds, AggregationContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException { AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData); super(name, type, config, context, parent, subFactoriesBuilder, metaData);
this.interval = interval; this.interval = interval;
@ -87,34 +60,13 @@ public final class DateHistogramAggregatorFactory
this.keyed = keyed; this.keyed = keyed;
this.minDocCount = minDocCount; this.minDocCount = minDocCount;
this.extendedBounds = extendedBounds; this.extendedBounds = extendedBounds;
this.rounding = rounding;
} }
public long minDocCount() { public long minDocCount() {
return minDocCount; return minDocCount;
} }
private Rounding createRounding() {
Rounding.Builder tzRoundingBuilder;
if (dateHistogramInterval != null) {
DateTimeUnit dateTimeUnit = DATE_FIELD_UNITS.get(dateHistogramInterval.toString());
if (dateTimeUnit != null) {
tzRoundingBuilder = Rounding.builder(dateTimeUnit);
} else {
// the interval is a time value?
tzRoundingBuilder = Rounding.builder(
TimeValue.parseTimeValue(dateHistogramInterval.toString(), null, getClass().getSimpleName() + ".interval"));
}
} else {
// the interval is an integer time value in millis?
tzRoundingBuilder = Rounding.builder(TimeValue.timeValueMillis(interval));
}
if (timeZone() != null) {
tzRoundingBuilder.timeZone(timeZone());
}
Rounding rounding = tzRoundingBuilder.build();
return rounding;
}
@Override @Override
protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket, protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, Aggregator parent, boolean collectsFromSingleBucket,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException { List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
@ -126,18 +78,7 @@ public final class DateHistogramAggregatorFactory
private Aggregator createAggregator(ValuesSource.Numeric valuesSource, Aggregator parent, List<PipelineAggregator> pipelineAggregators, private Aggregator createAggregator(ValuesSource.Numeric valuesSource, Aggregator parent, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException { Map<String, Object> metaData) throws IOException {
Rounding rounding = createRounding(); return new DateHistogramAggregator(name, factories, rounding, offset, order, keyed, minDocCount, extendedBounds, valuesSource,
// we need to round the bounds given by the user and we have to do it
// for every aggregator we create
// as the rounding is not necessarily an idempotent operation.
// todo we need to think of a better structure to the factory/agtor
// code so we won't need to do that
ExtendedBounds roundedBounds = null;
if (extendedBounds != null) {
// parse any string bounds to longs and round them
roundedBounds = extendedBounds.parseAndValidate(name, context.searchContext(), config.format()).round(rounding);
}
return new DateHistogramAggregator(name, factories, rounding, offset, order, keyed, minDocCount, roundedBounds, valuesSource,
config.format(), context, parent, pipelineAggregators, metaData); config.format(), context, parent, pipelineAggregators, metaData);
} }

View File

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

View File

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

View File

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

View File

@ -91,7 +91,6 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
: searcher.count(filter); : searcher.count(filter);
this.bucketCountThresholds = bucketCountThresholds; this.bucketCountThresholds = bucketCountThresholds;
this.significanceHeuristic = significanceHeuristic; this.significanceHeuristic = significanceHeuristic;
this.significanceHeuristic.initialize(context.searchContext());
setFieldInfo(); setFieldInfo();
} }

View File

@ -32,7 +32,6 @@ import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.Script.ScriptField; import org.elasticsearch.script.Script.ScriptField;
import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.support.XContentParseContext; import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
@ -49,7 +48,7 @@ public class ScriptHeuristic extends SignificanceHeuristic {
private final LongAccessor subsetDfHolder; private final LongAccessor subsetDfHolder;
private final LongAccessor supersetDfHolder; private final LongAccessor supersetDfHolder;
private final Script script; private final Script script;
ExecutableScript searchScript = null; ExecutableScript executableScript = null;
public ScriptHeuristic(Script script) { public ScriptHeuristic(Script script) {
subsetSizeHolder = new LongAccessor(); subsetSizeHolder = new LongAccessor();
@ -73,20 +72,20 @@ public class ScriptHeuristic extends SignificanceHeuristic {
@Override @Override
public void initialize(InternalAggregation.ReduceContext context) { public void initialize(InternalAggregation.ReduceContext context) {
initialize(context.scriptService()); initialize(context.scriptService().executable(script, ScriptContext.Standard.AGGS, Collections.emptyMap()));
} }
@Override @Override
public void initialize(SearchContext context) { public void initialize(SearchContext context) {
initialize(context.scriptService()); initialize(context.getQueryShardContext().getExecutableScript(script, ScriptContext.Standard.AGGS, Collections.emptyMap()));
} }
public void initialize(ScriptService scriptService) { public void initialize(ExecutableScript executableScript) {
searchScript = scriptService.executable(script, ScriptContext.Standard.AGGS, Collections.emptyMap()); this.executableScript = executableScript;
searchScript.setNextVar("_subset_freq", subsetDfHolder); this.executableScript.setNextVar("_subset_freq", subsetDfHolder);
searchScript.setNextVar("_subset_size", subsetSizeHolder); this.executableScript.setNextVar("_subset_size", subsetSizeHolder);
searchScript.setNextVar("_superset_freq", supersetDfHolder); this.executableScript.setNextVar("_superset_freq", supersetDfHolder);
searchScript.setNextVar("_superset_size", supersetSizeHolder); this.executableScript.setNextVar("_superset_size", supersetSizeHolder);
} }
/** /**
@ -100,7 +99,7 @@ public class ScriptHeuristic extends SignificanceHeuristic {
*/ */
@Override @Override
public double getScore(long subsetFreq, long subsetSize, long supersetFreq, long supersetSize) { 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(..). //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. // 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. // assertSearchResponse() might therefore fail although there is no problem.
@ -112,7 +111,7 @@ public class ScriptHeuristic extends SignificanceHeuristic {
supersetSizeHolder.value = supersetSize; supersetSizeHolder.value = supersetSize;
subsetDfHolder.value = subsetFreq; subsetDfHolder.value = subsetFreq;
supersetDfHolder.value = supersetFreq; supersetDfHolder.value = supersetFreq;
return ((Number) searchScript.run()).doubleValue(); return ((Number) executableScript.run()).doubleValue();
} }
@Override @Override
@ -171,26 +170,6 @@ public class ScriptHeuristic extends SignificanceHeuristic {
return new ScriptHeuristic(script); 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 final class LongAccessor extends Number {
public long value; public long value;
@Override @Override

View File

@ -26,18 +26,23 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.Script; 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.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import java.util.HashSet; import java.util.HashSet;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.function.Function;
public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder<ScriptedMetricAggregationBuilder> { public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder<ScriptedMetricAggregationBuilder> {
@ -182,9 +187,28 @@ public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder
@Override @Override
protected ScriptedMetricAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent, protected ScriptedMetricAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent,
Builder subfactoriesBuilder) throws IOException { Builder subfactoriesBuilder) throws IOException {
return new ScriptedMetricAggregatorFactory(name, type, initScript, mapScript, combineScript, reduceScript, params, context,
parent, subfactoriesBuilder, metaData); 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 @Override
protected XContentBuilder internalXContent(XContentBuilder builder, Params builderParams) throws IOException { protected XContentBuilder internalXContent(XContentBuilder builder, Params builderParams) throws IOException {

View File

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

View File

@ -19,7 +19,9 @@
package org.elasticsearch.search.aggregations.metrics.scripted; package org.elasticsearch.search.aggregations.metrics.scripted;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -34,22 +36,23 @@ import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.function.Function;
public class ScriptedMetricAggregatorFactory extends AggregatorFactory<ScriptedMetricAggregatorFactory> { public class ScriptedMetricAggregatorFactory extends AggregatorFactory<ScriptedMetricAggregatorFactory> {
private final Script initScript; private final Function<Map<String, Object>, SearchScript> mapScript;
private final Script mapScript; private final Function<Map<String, Object>, ExecutableScript> combineScript;
private final Script combineScript;
private final Script reduceScript; private final Script reduceScript;
private final Map<String, Object> params; 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, 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, Script reduceScript, Map<String, Object> params, AggregationContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException { AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData); super(name, type, context, parent, subFactories, metaData);
this.initScript = initScript;
this.mapScript = mapScript; this.mapScript = mapScript;
this.initScript = initScript;
this.combineScript = combineScript; this.combineScript = combineScript;
this.reduceScript = reduceScript; this.reduceScript = reduceScript;
this.params = params; this.params = params;
@ -68,16 +71,18 @@ public class ScriptedMetricAggregatorFactory extends AggregatorFactory<ScriptedM
params = new HashMap<>(); params = new HashMap<>();
params.put("_agg", new HashMap<String, Object>()); 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) { final ExecutableScript initScript = this.initScript.apply(params);
if (script == null) { final SearchScript mapScript = this.mapScript.apply(params);
return null; 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) { private static Script deepCopyScript(Script script, SearchContext context) {
@ -98,13 +103,13 @@ public class ScriptedMetricAggregatorFactory extends AggregatorFactory<ScriptedM
if (original instanceof Map) { if (original instanceof Map) {
Map<?, ?> originalMap = (Map<?, ?>) original; Map<?, ?> originalMap = (Map<?, ?>) original;
Map<Object, Object> clonedMap = new HashMap<>(); 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)); clonedMap.put(deepCopyParams(e.getKey(), context), deepCopyParams(e.getValue(), context));
} }
clone = (T) clonedMap; clone = (T) clonedMap;
} else if (original instanceof List) { } else if (original instanceof List) {
List<?> originalList = (List<?>) original; List<?> originalList = (List<?>) original;
List<Object> clonedList = new ArrayList<Object>(); List<Object> clonedList = new ArrayList<>();
for (Object o : originalList) { for (Object o : originalList) {
clonedList.add(deepCopyParams(o, context)); clonedList.add(deepCopyParams(o, context));
} }
@ -120,4 +125,5 @@ public class ScriptedMetricAggregatorFactory extends AggregatorFactory<ScriptedM
return clone; return clone;
} }
} }

View File

@ -530,20 +530,17 @@ public class TopHitsAggregationBuilder extends AbstractAggregationBuilder<TopHit
@Override @Override
protected TopHitsAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subfactoriesBuilder) protected TopHitsAggregatorFactory doBuild(AggregationContext context, AggregatorFactory<?> parent, Builder subfactoriesBuilder)
throws IOException { throws IOException {
List<ScriptFieldsContext.ScriptField> scriptFields = null; List<ScriptFieldsContext.ScriptField> fields = new ArrayList<>();
if (this.scriptFields != null) { if (scriptFields != null) {
scriptFields = new ArrayList<>(); for (ScriptField field : scriptFields) {
for (ScriptField field : this.scriptFields) { SearchScript searchScript = context.searchContext().getQueryShardContext().getSearchScript(field.script(),
SearchScript searchScript = context.searchContext().scriptService().search( ScriptContext.Standard.SEARCH, Collections.emptyMap());
context.searchContext().lookup(), field.script(), ScriptContext.Standard.SEARCH, Collections.emptyMap()); fields.add(new org.elasticsearch.search.fetch.subphase.ScriptFieldsContext.ScriptField(
scriptFields.add(new ScriptFieldsContext.ScriptField(
field.fieldName(), searchScript, field.ignoreFailure())); field.fieldName(), searchScript, field.ignoreFailure()));
} }
} else {
scriptFields = Collections.emptyList();
} }
return new TopHitsAggregatorFactory(name, type, from, size, explain, version, trackScores, sorts, highlightBuilder, 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); parent, subfactoriesBuilder, metaData);
} }

View File

@ -56,8 +56,8 @@ public class TopHitsAggregatorFactory extends AggregatorFactory<TopHitsAggregato
public TopHitsAggregatorFactory(String name, Type type, int from, int size, boolean explain, boolean version, boolean trackScores, public TopHitsAggregatorFactory(String name, Type type, int from, int size, boolean explain, boolean version, boolean trackScores,
List<SortBuilder<?>> sorts, HighlightBuilder highlightBuilder, StoredFieldsContext storedFieldsContext, List<SortBuilder<?>> sorts, HighlightBuilder highlightBuilder, StoredFieldsContext storedFieldsContext,
List<String> docValueFields, List<ScriptFieldsContext.ScriptField> scriptFields, FetchSourceContext fetchSourceContext, List<String> docValueFields, List<ScriptFieldsContext.ScriptField> scriptFields, FetchSourceContext fetchSourceContext,
AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactories, AggregationContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactories, Map<String, Object> metaData)
Map<String, Object> metaData) throws IOException { throws IOException {
super(name, type, context, parent, subFactories, metaData); super(name, type, context, parent, subFactories, metaData);
this.from = from; this.from = from;
this.size = size; this.size = size;

View File

@ -32,6 +32,7 @@ import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregatorFactory; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregatorFactory; import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregatorFactory;
import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder;
@ -141,7 +142,7 @@ public class DerivativePipelineAggregationBuilder extends AbstractPipelineAggreg
} }
Long xAxisUnits = null; Long xAxisUnits = null;
if (units != null) { if (units != null) {
DateTimeUnit dateTimeUnit = DateHistogramAggregatorFactory.DATE_FIELD_UNITS.get(units); DateTimeUnit dateTimeUnit = DateHistogramAggregationBuilder.DATE_FIELD_UNITS.get(units);
if (dateTimeUnit != null) { if (dateTimeUnit != null) {
xAxisUnits = dateTimeUnit.field(DateTimeZone.UTC).getDurationField().getUnitMillis(); xAxisUnits = dateTimeUnit.field(DateTimeZone.UTC).getDurationField().getUnitMillis();
} else { } else {

View File

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

View File

@ -376,8 +376,11 @@ public abstract class ValuesSourceAggregationBuilder<VS extends ValuesSource, AB
} }
private SearchScript createScript(Script script, SearchContext context) { private SearchScript createScript(Script script, SearchContext context) {
return script == null ? null if (script == null) {
: context.scriptService().search(context.lookup(), script, ScriptContext.Standard.AGGS, Collections.emptyMap()); return null;
} else {
return context.getQueryShardContext().getSearchScript(script, ScriptContext.Standard.AGGS, Collections.emptyMap());
}
} }
private static DocValueFormat resolveFormat(@Nullable String format, @Nullable ValueType valueType) { private static DocValueFormat resolveFormat(@Nullable String format, @Nullable ValueType valueType) {

View File

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

View File

@ -42,7 +42,6 @@ import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchExtBuilder; import org.elasticsearch.search.SearchExtBuilder;
import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.SearchContextAggregations; import org.elasticsearch.search.aggregations.SearchContextAggregations;
@ -118,8 +117,6 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
} }
} }
private boolean nowInMillisUsed;
@Override @Override
protected final void closeInternal() { protected final void closeInternal() {
try { try {
@ -162,21 +159,6 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
public abstract long getOriginNanoTime(); public abstract long getOriginNanoTime();
public final long nowInMillis() {
nowInMillisUsed = true;
return nowInMillisImpl();
}
public final boolean nowInMillisUsed() {
return nowInMillisUsed;
}
public final void resetNowInMillisUsed() {
this.nowInMillisUsed = false;
}
protected abstract long nowInMillisImpl();
public abstract ScrollContext scrollContext(); public abstract ScrollContext scrollContext();
public abstract SearchContext scrollContext(ScrollContext scroll); public abstract SearchContext scrollContext(ScrollContext scroll);
@ -238,8 +220,6 @@ public abstract class SearchContext extends AbstractRefCounted implements Releas
public abstract SimilarityService similarityService(); public abstract SimilarityService similarityService();
public abstract ScriptService scriptService();
public abstract BigArrays bigArrays(); public abstract BigArrays bigArrays();
public abstract BitsetFilterCache bitsetFilterCache(); public abstract BitsetFilterCache bitsetFilterCache();

View File

@ -55,9 +55,7 @@ import org.elasticsearch.search.MultiValueMode;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.Locale; import java.util.Locale;
import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.Optional; import java.util.Optional;
@ -286,8 +284,7 @@ public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
@Override @Override
public SortFieldAndFormat build(QueryShardContext context) throws IOException { public SortFieldAndFormat build(QueryShardContext context) throws IOException {
final SearchScript searchScript = context.getScriptService().search( final SearchScript searchScript = context.getSearchScript(script, ScriptContext.Standard.SEARCH, Collections.emptyMap());
context.lookup(), script, ScriptContext.Standard.SEARCH, Collections.emptyMap());
MultiValueMode valueMode = null; MultiValueMode valueMode = null;
if (sortMode != null) { if (sortMode != null) {

View File

@ -55,6 +55,7 @@ import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.function.Function;
public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> { public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
private final BytesRef SEPARATOR = new BytesRef(" "); private final BytesRef SEPARATOR = new BytesRef(" ");
@ -109,7 +110,7 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
response.addTerm(resultEntry); response.addTerm(resultEntry);
final BytesRefBuilder byteSpare = new BytesRefBuilder(); 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(); final boolean collatePrune = (collateScript != null) && suggestion.collatePrune();
for (int i = 0; i < checkerResult.corrections.length; i++) { for (int i = 0; i < checkerResult.corrections.length; i++) {
Correction correction = checkerResult.corrections[i]; Correction correction = checkerResult.corrections[i];
@ -121,8 +122,7 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
final Map<String, Object> vars = suggestion.getCollateScriptParams(); final Map<String, Object> vars = suggestion.getCollateScriptParams();
vars.put(SUGGESTION_TEMPLATE_VAR_NAME, spare.toString()); vars.put(SUGGESTION_TEMPLATE_VAR_NAME, spare.toString());
QueryShardContext shardContext = suggestion.getShardContext(); QueryShardContext shardContext = suggestion.getShardContext();
ScriptService scriptService = shardContext.getScriptService(); final ExecutableScript executable = collateScript.apply(vars);
final ExecutableScript executable = scriptService.executable(collateScript, vars);
final BytesReference querySource = (BytesReference) executable.run(); final BytesReference querySource = (BytesReference) executable.run();
try (XContentParser parser = XContentFactory.xContent(querySource).createParser(querySource)) { try (XContentParser parser = XContentFactory.xContent(querySource).createParser(querySource)) {
Optional<QueryBuilder> innerQueryBuilder = shardContext.newParseContext(parser).parseInnerQueryBuilder(); 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.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.script.CompiledScript; import org.elasticsearch.script.CompiledScript;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptService;
@ -56,6 +57,7 @@ import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.Objects; import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.function.Function;
/** /**
* Defines the actual suggest command for phrase suggestions ( <tt>phrase</tt>). * Defines the actual suggest command for phrase suggestions ( <tt>phrase</tt>).
@ -633,8 +635,8 @@ public class PhraseSuggestionBuilder extends SuggestionBuilder<PhraseSuggestionB
} }
if (this.collateQuery != null) { if (this.collateQuery != null) {
CompiledScript compiledScript = context.getScriptService().compile(this.collateQuery, ScriptContext.Standard.SEARCH, Function<Map<String, Object>, ExecutableScript> compiledScript = context.getLazyExecutableScript(this.collateQuery,
Collections.emptyMap()); ScriptContext.Standard.SEARCH, Collections.emptyMap());
suggestionContext.setCollateQueryScript(compiledScript); suggestionContext.setCollateQueryScript(compiledScript);
if (this.collateParams != null) { if (this.collateParams != null) {
suggestionContext.setCollateScriptParams(this.collateParams); suggestionContext.setCollateScriptParams(this.collateParams);

View File

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

View File

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

View File

@ -46,9 +46,10 @@ public class QueryShardContextTests extends ESTestCase {
IndexSettings indexSettings = new IndexSettings(indexMetadata.build(), Settings.EMPTY); IndexSettings indexSettings = new IndexSettings(indexMetadata.build(), Settings.EMPTY);
MapperService mapperService = mock(MapperService.class); MapperService mapperService = mock(MapperService.class);
when(mapperService.getIndexSettings()).thenReturn(indexSettings); when(mapperService.getIndexSettings()).thenReturn(indexSettings);
final long nowInMillis = randomPositiveLong();
QueryShardContext context = new QueryShardContext( QueryShardContext context = new QueryShardContext(
indexSettings, null, null, mapperService, null, null, null, null, null, null indexSettings, null, null, mapperService, null, null, null, null, null, null,
); () -> nowInMillis);
context.setAllowUnmappedFields(false); context.setAllowUnmappedFields(false);
MappedFieldType fieldType = new TextFieldMapper.TextFieldType(); MappedFieldType fieldType = new TextFieldMapper.TextFieldType();

View File

@ -89,4 +89,9 @@ public class ScriptQueryBuilderTests extends AbstractQueryTestCase<ScriptQueryBu
//adding additional objects within the params object. //adding additional objects within the params object.
return Collections.singleton(Script.ScriptField.PARAMS.getPreferredName()); return Collections.singleton(Script.ScriptField.PARAMS.getPreferredName());
} }
@Override
protected boolean isCachable(ScriptQueryBuilder queryBuilder) {
return false;
}
} }

View File

@ -172,6 +172,7 @@ public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase<Functi
break; break;
case 3: case 3:
RandomScoreFunctionBuilder randomScoreFunctionBuilder = new RandomScoreFunctionBuilderWithFixedSeed(); RandomScoreFunctionBuilder randomScoreFunctionBuilder = new RandomScoreFunctionBuilderWithFixedSeed();
if (randomBoolean()) { // sometimes provide no seed
if (randomBoolean()) { if (randomBoolean()) {
randomScoreFunctionBuilder.seed(randomLong()); randomScoreFunctionBuilder.seed(randomLong());
} else if (randomBoolean()) { } else if (randomBoolean()) {
@ -179,6 +180,7 @@ public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase<Functi
} else { } else {
randomScoreFunctionBuilder.seed(randomAsciiOfLengthBetween(1, 10)); randomScoreFunctionBuilder.seed(randomAsciiOfLengthBetween(1, 10));
} }
}
functionBuilder = randomScoreFunctionBuilder; functionBuilder = randomScoreFunctionBuilder;
break; break;
default: default:
@ -777,7 +779,9 @@ public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase<Functi
throws IOException, ParsingException { throws IOException, ParsingException {
RandomScoreFunctionBuilder builder = RandomScoreFunctionBuilder.fromXContent(parseContext); RandomScoreFunctionBuilder builder = RandomScoreFunctionBuilder.fromXContent(parseContext);
RandomScoreFunctionBuilderWithFixedSeed replacement = new RandomScoreFunctionBuilderWithFixedSeed(); RandomScoreFunctionBuilderWithFixedSeed replacement = new RandomScoreFunctionBuilderWithFixedSeed();
if (builder.getSeed() != null) {
replacement.seed(builder.getSeed()); replacement.seed(builder.getSeed());
}
return replacement; return replacement;
} }
} }
@ -789,4 +793,18 @@ public class FunctionScoreQueryBuilderTests extends AbstractQueryTestCase<Functi
RandomScoreFunctionBuilderWithFixedSeed::new, RandomScoreFunctionBuilderWithFixedSeed::fromXContent)); RandomScoreFunctionBuilderWithFixedSeed::new, RandomScoreFunctionBuilderWithFixedSeed::fromXContent));
} }
} }
@Override
protected boolean isCachable(FunctionScoreQueryBuilder queryBuilder) {
FilterFunctionBuilder[] filterFunctionBuilders = queryBuilder.filterFunctionBuilders();
for (FilterFunctionBuilder builder : filterFunctionBuilders) {
if (builder.getScoreFunction() instanceof ScriptScoreFunctionBuilder) {
return false;
} else if (builder.getScoreFunction() instanceof RandomScoreFunctionBuilder
&& ((RandomScoreFunctionBuilder) builder.getScoreFunction()).getSeed() == null) {
return false;
}
}
return true;
}
} }

View File

@ -1203,4 +1203,46 @@ public class DateHistogramIT extends ESIntegTestCase {
assertThat(((DateTime) buckets.get(2).getKey()).getMillis() - ((DateTime) buckets.get(1).getKey()).getMillis(), equalTo(3600000L)); assertThat(((DateTime) buckets.get(2).getKey()).getMillis() - ((DateTime) buckets.get(1).getKey()).getMillis(), equalTo(3600000L));
assertThat(((DateTime) buckets.get(3).getKey()).getMillis() - ((DateTime) buckets.get(2).getKey()).getMillis(), equalTo(3600000L)); assertThat(((DateTime) buckets.get(3).getKey()).getMillis() - ((DateTime) buckets.get(2).getKey()).getMillis(), equalTo(3600000L));
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=date")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("d", date(1, 1)),
client().prepareIndex("cache_test_idx", "type", "2").setSource("d", date(2, 1)));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
Map<String, Object> params = new HashMap<>();
params.put("fieldname", "d");
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(dateHistogram("histo").field("d")
.script(new Script(DateScriptMocks.PlusOneMonthScript.NAME, ScriptType.INLINE, "native", params))
.dateHistogramInterval(DateHistogramInterval.MONTH)).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(dateHistogram("histo").field("d").dateHistogramInterval(DateHistogramInterval.MONTH)).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.script.ScriptService.ScriptType;
@ -866,4 +867,51 @@ public class DateRangeIT extends ESIntegTestCase {
assertThat(buckets.get(0).getDocCount(), equalTo(0L)); assertThat(buckets.get(0).getDocCount(), equalTo(0L));
assertThat(buckets.get(0).getAggregations().asList().isEmpty(), is(true)); assertThat(buckets.get(0).getAggregations().asList().isEmpty(), is(true));
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "date", "type=date")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true,
client().prepareIndex("cache_test_idx", "type", "1")
.setSource(jsonBuilder().startObject().field("date", date(1, 1)).endObject()),
client().prepareIndex("cache_test_idx", "type", "2")
.setSource(jsonBuilder().startObject().field("date", date(2, 1)).endObject()));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
Map<String, Object> params = new HashMap<>();
params.put("fieldname", "date");
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(dateRange("foo").field("date")
.script(new Script(DateScriptMocks.PlusOneMonthScript.NAME, ScriptType.INLINE, "native", params))
.addRange(new DateTime(2012, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC), new DateTime(2013, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC)))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(dateRange("foo").field("date")
.addRange(new DateTime(2012, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC), new DateTime(2013, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC)))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -21,13 +21,16 @@ package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.fielddata.ScriptDocValues; import org.elasticsearch.index.fielddata.ScriptDocValues;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.ScoreAccessor; import org.elasticsearch.script.ScoreAccessor;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.search.aggregations.AggregationTestScriptsPlugin; import org.elasticsearch.search.aggregations.AggregationTestScriptsPlugin;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.bucket.LongTermsIT.CustomScriptPlugin;
import org.elasticsearch.search.aggregations.bucket.filter.Filter; import org.elasticsearch.search.aggregations.bucket.filter.Filter;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
import org.elasticsearch.search.aggregations.bucket.terms.Terms; import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@ -56,7 +59,6 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery; import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.scriptFunction; import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.scriptFunction;
import static org.elasticsearch.script.ScriptService.ScriptType;
import static org.elasticsearch.search.aggregations.AggregationBuilders.avg; import static org.elasticsearch.search.aggregations.AggregationBuilders.avg;
import static org.elasticsearch.search.aggregations.AggregationBuilders.extendedStats; import static org.elasticsearch.search.aggregations.AggregationBuilders.extendedStats;
import static org.elasticsearch.search.aggregations.AggregationBuilders.filter; import static org.elasticsearch.search.aggregations.AggregationBuilders.filter;
@ -1179,4 +1181,42 @@ public class DoubleTermsIT extends AbstractTermsTestCase {
public void testOtherDocCount() { public void testOtherDocCount() {
testOtherDocCount(SINGLE_VALUED_FIELD_NAME, MULTI_VALUED_FIELD_NAME); testOtherDocCount(SINGLE_VALUED_FIELD_NAME, MULTI_VALUED_FIELD_NAME);
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=float")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1.5),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2.5));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(
terms("terms").field("d").script(new Script("_value + 1", ScriptType.INLINE, CustomScriptPlugin.NAME, null))).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(terms("terms").field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations.bucket;
import com.carrotsearch.hppc.LongHashSet; import com.carrotsearch.hppc.LongHashSet;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.MockScriptPlugin; import org.elasticsearch.script.MockScriptPlugin;
@ -995,4 +996,43 @@ public class HistogramIT extends ESIntegTestCase {
assertEquals(0.05, (double) buckets.get(1).getKey(), 0.01d); assertEquals(0.05, (double) buckets.get(1).getKey(), 0.01d);
assertEquals(1, buckets.get(1).getDocCount()); assertEquals(1, buckets.get(1).getDocCount());
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=float")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("d", -0.6),
client().prepareIndex("cache_test_idx", "type", "2").setSource("d", 0.1));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(histogram("histo").field("d")
.script(new Script("_value + 1", ScriptType.INLINE, CustomScriptPlugin.NAME, emptyMap())).interval(0.7).offset(0.05)).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(histogram("histo").field("d").interval(0.7).offset(0.05))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -21,12 +21,15 @@ package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.fielddata.ScriptDocValues; import org.elasticsearch.index.fielddata.ScriptDocValues;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.search.aggregations.AggregationTestScriptsPlugin; import org.elasticsearch.search.aggregations.AggregationTestScriptsPlugin;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.bucket.StringTermsIT.CustomScriptPlugin;
import org.elasticsearch.search.aggregations.bucket.filter.Filter; import org.elasticsearch.search.aggregations.bucket.filter.Filter;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
import org.elasticsearch.search.aggregations.bucket.terms.Terms; import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@ -53,7 +56,6 @@ import java.util.function.Function;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.script.ScriptService.ScriptType;
import static org.elasticsearch.search.aggregations.AggregationBuilders.avg; import static org.elasticsearch.search.aggregations.AggregationBuilders.avg;
import static org.elasticsearch.search.aggregations.AggregationBuilders.extendedStats; import static org.elasticsearch.search.aggregations.AggregationBuilders.extendedStats;
import static org.elasticsearch.search.aggregations.AggregationBuilders.filter; import static org.elasticsearch.search.aggregations.AggregationBuilders.filter;
@ -1136,4 +1138,42 @@ public class LongTermsIT extends AbstractTermsTestCase {
public void testOtherDocCount() { public void testOtherDocCount() {
testOtherDocCount(SINGLE_VALUED_FIELD_NAME, MULTI_VALUED_FIELD_NAME); testOtherDocCount(SINGLE_VALUED_FIELD_NAME, MULTI_VALUED_FIELD_NAME);
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(
terms("terms").field("d").script(new Script("_value + 1", ScriptType.INLINE, CustomScriptPlugin.NAME, null))).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(terms("terms").field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -20,9 +20,11 @@ package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.fielddata.ScriptDocValues; import org.elasticsearch.index.fielddata.ScriptDocValues;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.search.aggregations.AggregationTestScriptsPlugin; import org.elasticsearch.search.aggregations.AggregationTestScriptsPlugin;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
@ -32,21 +34,25 @@ import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.metrics.sum.Sum; import org.elasticsearch.search.aggregations.metrics.sum.Sum;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.hamcrest.Matchers; import org.hamcrest.Matchers;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.function.Function; import java.util.function.Function;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.script.ScriptService.ScriptType; import static org.elasticsearch.search.aggregations.AggregationBuilders.dateRange;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.range; import static org.elasticsearch.search.aggregations.AggregationBuilders.range;
import static org.elasticsearch.search.aggregations.AggregationBuilders.sum; import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.is;
@ -907,4 +913,46 @@ public class RangeIT extends ESIntegTestCase {
assertThat(buckets.get(0).getDocCount(), equalTo(0L)); assertThat(buckets.get(0).getDocCount(), equalTo(0L));
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "i", "type=integer")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true,
client().prepareIndex("cache_test_idx", "type", "1").setSource(jsonBuilder().startObject().field("i", 1).endObject()),
client().prepareIndex("cache_test_idx", "type", "2").setSource(jsonBuilder().startObject().field("i", 2).endObject()));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
Map<String, Object> params = new HashMap<>();
params.put("fieldname", "date");
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(
range("foo").field("i").script(new Script("_value + 1", ScriptType.INLINE, CustomScriptPlugin.NAME, null)).addRange(0, 10))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(range("foo").field("i").addRange(0, 10)).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -22,6 +22,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
@ -547,4 +548,43 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
SharedSignificantTermsTestMethods.aggregateAndCheckFromSeveralShards(this); SharedSignificantTermsTestMethods.aggregateAndCheckFromSeveralShards(this);
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
ScriptHeuristic scriptHeuristic = getScriptSignificanceHeuristic();
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(significantTerms("foo").field("s").significanceHeuristic(scriptHeuristic)).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(significantTerms("foo").field("s")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -24,11 +24,13 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.fielddata.ScriptDocValues; import org.elasticsearch.index.fielddata.ScriptDocValues;
import org.elasticsearch.index.mapper.IndexFieldMapper; import org.elasticsearch.index.mapper.IndexFieldMapper;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.AggregationTestScriptsPlugin; import org.elasticsearch.search.aggregations.AggregationTestScriptsPlugin;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
@ -61,7 +63,6 @@ import java.util.function.Function;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery; import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.script.ScriptService.ScriptType;
import static org.elasticsearch.search.aggregations.AggregationBuilders.avg; import static org.elasticsearch.search.aggregations.AggregationBuilders.avg;
import static org.elasticsearch.search.aggregations.AggregationBuilders.count; import static org.elasticsearch.search.aggregations.AggregationBuilders.count;
import static org.elasticsearch.search.aggregations.AggregationBuilders.extendedStats; import static org.elasticsearch.search.aggregations.AggregationBuilders.extendedStats;
@ -1511,4 +1512,44 @@ public class StringTermsIT extends AbstractTermsTestCase {
public void testOtherDocCount() { public void testOtherDocCount() {
testOtherDocCount(SINGLE_VALUED_FIELD_NAME, MULTI_VALUED_FIELD_NAME); testOtherDocCount(SINGLE_VALUED_FIELD_NAME, MULTI_VALUED_FIELD_NAME);
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=keyword")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", "foo"),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", "bar"));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(
terms("terms").field("d").script(new Script("'foo_' + _value", ScriptType.INLINE, CustomScriptPlugin.NAME, null)))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(terms("terms").field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

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

View File

@ -56,7 +56,9 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.filter;
import static org.elasticsearch.search.aggregations.AggregationBuilders.global; import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.notNullValue;
@ -353,6 +355,44 @@ public class AvgIT extends AbstractNumericTestCase {
} }
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(avg("foo").field("d").script(new Script("", ScriptType.INLINE, FieldValueScriptEngine.NAME, null))).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(avg("foo").field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
/** /**
* Mock plugin for the {@link ExtractFieldScriptEngine} * Mock plugin for the {@link ExtractFieldScriptEngine}
*/ */

View File

@ -26,6 +26,7 @@ import org.elasticsearch.index.fielddata.ScriptDocValues;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.MockScriptPlugin; import org.elasticsearch.script.MockScriptPlugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.bucket.global.Global; import org.elasticsearch.search.aggregations.bucket.global.Global;
import org.elasticsearch.search.aggregations.bucket.terms.Terms; import org.elasticsearch.search.aggregations.bucket.terms.Terms;
@ -41,10 +42,10 @@ import java.util.function.Function;
import static java.util.Collections.emptyMap; import static java.util.Collections.emptyMap;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.script.ScriptService.ScriptType;
import static org.elasticsearch.search.aggregations.AggregationBuilders.cardinality; import static org.elasticsearch.search.aggregations.AggregationBuilders.cardinality;
import static org.elasticsearch.search.aggregations.AggregationBuilders.global; import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThan;
@ -445,4 +446,44 @@ public class CardinalityIT extends ESIntegTestCase {
assertCount(count, 2); assertCount(count, 2);
} }
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(
cardinality("foo").field("d").script(new Script("_value", ScriptType.INLINE, CustomScriptPlugin.NAME, emptyMap())))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(cardinality("foo").field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics; package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.script.ScriptService.ScriptType;
@ -46,7 +47,9 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.missing; import static org.elasticsearch.search.aggregations.AggregationBuilders.missing;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.notNullValue;
@ -630,4 +633,44 @@ public class ExtendedStatsIT extends AbstractNumericTestCase {
assertThat(stats.getStdDeviationBound(ExtendedStats.Bounds.LOWER), equalTo(stats.getAvg() - (stats.getStdDeviation() * sigma))); assertThat(stats.getStdDeviationBound(ExtendedStats.Bounds.LOWER), equalTo(stats.getAvg() - (stats.getStdDeviation() * sigma)));
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(extendedStats("foo").field("d")
.script(new Script("_value + 1", ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, null)))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(extendedStats("foo").field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.script.ScriptService.ScriptType;
@ -49,7 +50,9 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.percentileRanks; import static org.elasticsearch.search.aggregations.AggregationBuilders.percentileRanks;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo;
@ -530,4 +533,45 @@ public class HDRPercentileRanksIT extends AbstractNumericTestCase {
} }
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client()
.prepareSearch("cache_test_idx").setSize(0).addAggregation(percentileRanks("foo").method(PercentilesMethod.HDR).field("d")
.values(50.0).script(new Script("_value - 1", ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, emptyMap())))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(percentileRanks("foo").method(PercentilesMethod.HDR).field("d").values(50.0)).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
@ -49,7 +50,9 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.percentiles; import static org.elasticsearch.search.aggregations.AggregationBuilders.percentiles;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.closeTo;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@ -521,4 +524,45 @@ public class HDRPercentilesIT extends AbstractNumericTestCase {
} }
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(percentiles("foo").method(PercentilesMethod.HDR).field("d").percentiles(50.0)
.script(new Script("_value - 1", ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, emptyMap())))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(percentiles("foo").method(PercentilesMethod.HDR).field("d").percentiles(50.0)).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics; package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.script.ScriptService.ScriptType;
@ -44,7 +45,9 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.max; import static org.elasticsearch.search.aggregations.AggregationBuilders.max;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.notNullValue;
@ -352,4 +355,43 @@ public class MaxIT extends AbstractNumericTestCase {
} }
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(
max("foo").field("d").script(new Script("_value + 1", ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, emptyMap())))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(max("foo").field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.metrics; package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.script.ScriptService.ScriptType;
@ -44,7 +45,9 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.min; import static org.elasticsearch.search.aggregations.AggregationBuilders.min;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.notNullValue;
@ -365,4 +368,43 @@ public class MinIT extends AbstractNumericTestCase {
} }
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(
min("foo").field("d").script(new Script("_value - 1", ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, emptyMap())))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(min("foo").field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -933,4 +933,33 @@ public class ScriptedMetricIT extends ESIntegTestCase {
assertThat(aggregationResult.size(), equalTo(1)); assertThat(aggregationResult.size(), equalTo(1));
assertThat(aggregationResult.get(0), equalTo(0)); assertThat(aggregationResult.get(0), equalTo(0));
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
Script mapScript = new Script("_agg['count'] = 1", ScriptType.INLINE, CustomScriptPlugin.NAME, null);
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(scriptedMetric("foo").mapScript(mapScript)).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
}
} }

View File

@ -22,6 +22,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier; import org.apache.logging.log4j.util.Supplier;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.script.ScriptService.ScriptType;
@ -47,7 +48,9 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.stats; import static org.elasticsearch.search.aggregations.AggregationBuilders.stats;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.notNullValue;
@ -485,4 +488,42 @@ public class StatsIT extends AbstractNumericTestCase {
} }
assertThat("Not all shards are initialized", response.getSuccessfulShards(), equalTo(response.getTotalShards())); assertThat("Not all shards are initialized", response.getSuccessfulShards(), equalTo(response.getTotalShards()));
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(
stats("foo").field("d").script(new Script("_value + 1", ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, null))).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(stats("foo").field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -55,7 +55,9 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.sum; import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.notNullValue;
@ -347,6 +349,44 @@ public class SumIT extends AbstractNumericTestCase {
} }
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(sum("foo").field("d").script(new Script("", ScriptType.INLINE, FieldValueScriptEngine.NAME, null))).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(sum("foo").field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
/** /**
* Mock plugin for the {@link ExtractFieldScriptEngine} * Mock plugin for the {@link ExtractFieldScriptEngine}
*/ */

View File

@ -20,10 +20,9 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.index.fielddata.ScriptDocValues;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.MockScriptPlugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.search.aggregations.AggregationTestScriptsPlugin; import org.elasticsearch.search.aggregations.AggregationTestScriptsPlugin;
@ -43,8 +42,6 @@ import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.function.Function;
import static java.util.Collections.emptyMap; import static java.util.Collections.emptyMap;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery; import static org.elasticsearch.index.query.QueryBuilders.termQuery;
@ -53,7 +50,9 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.percentileRanks; import static org.elasticsearch.search.aggregations.AggregationBuilders.percentileRanks;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo;
@ -484,4 +483,42 @@ public class TDigestPercentileRanksIT extends AbstractNumericTestCase {
} }
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(percentileRanks("foo").field("d").values(50.0)
.script(new Script("_value - 1", ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, emptyMap()))).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(percentileRanks("foo").field("d").values(50.0)).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
@ -50,7 +51,9 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.percentiles; import static org.elasticsearch.search.aggregations.AggregationBuilders.percentiles;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo;
@ -465,4 +468,43 @@ public class TDigestPercentilesIT extends AbstractNumericTestCase {
} }
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(percentiles("foo").field("d")
.percentiles(50.0).script(new Script("_value - 1", ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, emptyMap())))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(percentiles("foo").field("d").percentiles(50.0)).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -25,6 +25,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
@ -33,6 +34,7 @@ import org.elasticsearch.script.MockScriptEngine;
import org.elasticsearch.script.MockScriptPlugin; import org.elasticsearch.script.MockScriptPlugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptService;
import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
@ -993,4 +995,42 @@ public class TopHitsIT extends ESIntegTestCase {
} }
} }
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(topHits("foo").scriptField("bar", new Script("5", ScriptType.INLINE, CustomScriptPlugin.NAME, null))).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(topHits("foo")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
} }

View File

@ -28,9 +28,7 @@ import org.elasticsearch.script.CompiledScript;
import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.LeafSearchScript; import org.elasticsearch.script.LeafSearchScript;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptEngineRegistry;
import org.elasticsearch.script.ScriptEngineService; import org.elasticsearch.script.ScriptEngineService;
import org.elasticsearch.script.ScriptModule;
import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.script.SearchScript; import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.aggregations.bucket.global.Global; import org.elasticsearch.search.aggregations.bucket.global.Global;
@ -43,14 +41,15 @@ import java.io.IOException;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.search.aggregations.AggregationBuilders.count; import static org.elasticsearch.search.aggregations.AggregationBuilders.count;
import static org.elasticsearch.search.aggregations.AggregationBuilders.global; import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.notNullValue;
@ -209,6 +208,45 @@ public class ValueCountIT extends ESIntegTestCase {
assertThat(valueCount.getValue(), equalTo(20L)); assertThat(valueCount.getValue(), equalTo(20L));
} }
/**
* Make sure that a request using a script does not get cached and a request
* not using a script does get cached.
*/
public void testDontCacheScripts() throws Exception {
assertAcked(prepareCreate("cache_test_idx").addMapping("type", "d", "type=long")
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get());
indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1").setSource("s", 1),
client().prepareIndex("cache_test_idx", "type", "2").setSource("s", 2));
// Make sure we are starting with a clear cache
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// Test that a request using a script does not get cached
SearchResponse r = client().prepareSearch("cache_test_idx").setSize(0)
.addAggregation(count("foo").field("d").script(new Script("value", ScriptType.INLINE, FieldValueScriptEngine.NAME, null)))
.get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(0L));
// To make sure that the cache is working test that a request not using
// a script is cached
r = client().prepareSearch("cache_test_idx").setSize(0).addAggregation(count("foo").field("d")).get();
assertSearchResponse(r);
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getHitCount(), equalTo(0L));
assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache()
.getMissCount(), equalTo(1L));
}
/** /**
* Mock plugin for the {@link FieldValueScriptEngine} * Mock plugin for the {@link FieldValueScriptEngine}
*/ */

View File

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

View File

@ -156,12 +156,13 @@ public class QueryRescoreBuilderTests extends ESTestCase {
* than the test builder * than the test builder
*/ */
public void testBuildRescoreSearchContext() throws ElasticsearchParseException, IOException { public void testBuildRescoreSearchContext() throws ElasticsearchParseException, IOException {
final long nowInMillis = randomPositiveLong();
Settings indexSettings = Settings.builder() Settings indexSettings = Settings.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAsciiOfLengthBetween(1, 10), indexSettings); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAsciiOfLengthBetween(1, 10), indexSettings);
// shard context will only need indicesQueriesRegistry for building Query objects nested in query rescorer // 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, QueryShardContext mockShardContext = new QueryShardContext(idxSettings, null, null, null, null, null, indicesQueriesRegistry,
null, null, null) { null, null, null, () -> nowInMillis) {
@Override @Override
public MappedFieldType fieldMapper(String name) { public MappedFieldType fieldMapper(String name) {
TextFieldMapper.Builder builder = new TextFieldMapper.Builder(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.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
@ -235,8 +234,9 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
public void onCache(ShardId shardId, Accountable accountable) { public void onCache(ShardId shardId, Accountable accountable) {
} }
}); });
long nowInMillis = randomPositiveLong();
return new QueryShardContext(idxSettings, bitsetFilterCache, ifds, null, null, scriptService, return new QueryShardContext(idxSettings, bitsetFilterCache, ifds, null, null, scriptService,
indicesQueriesRegistry, null, null, null) { indicesQueriesRegistry, null, null, null, () -> nowInMillis) {
@Override @Override
public MappedFieldType fieldMapper(String name) { public MappedFieldType fieldMapper(String name) {
return provideMappedFieldType(name); return provideMappedFieldType(name);

View File

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

View File

@ -366,6 +366,9 @@ public class PercolateQueryBuilder extends AbstractQueryBuilder<PercolateQueryBu
@Override @Override
protected Query doToQuery(QueryShardContext context) throws IOException { protected Query doToQuery(QueryShardContext context) throws IOException {
// Call nowInMillis() so that this query becomes un-cacheable since we
// can't be sure that it doesn't use now or scripts
context.nowInMillis();
if (indexedDocumentIndex != null || indexedDocumentType != null || indexedDocumentId != null) { if (indexedDocumentIndex != null || indexedDocumentType != null || indexedDocumentId != null) {
throw new IllegalStateException("query builder must be rewritten first"); throw new IllegalStateException("query builder must be rewritten first");
} }
@ -407,6 +410,9 @@ public class PercolateQueryBuilder extends AbstractQueryBuilder<PercolateQueryBu
Version indexVersionCreated = context.getIndexSettings().getIndexVersionCreated(); Version indexVersionCreated = context.getIndexSettings().getIndexVersionCreated();
boolean mapUnmappedFieldsAsString = context.getIndexSettings() boolean mapUnmappedFieldsAsString = context.getIndexSettings()
.getValue(PercolatorFieldMapper.INDEX_MAP_UNMAPPED_FIELDS_AS_STRING_SETTING); .getValue(PercolatorFieldMapper.INDEX_MAP_UNMAPPED_FIELDS_AS_STRING_SETTING);
// We have to make a copy of the QueryShardContext here so we can have a unfrozen version for parsing the legacy
// percolator queries
QueryShardContext percolateShardContext = new QueryShardContext(context);
if (indexVersionCreated.onOrAfter(Version.V_5_0_0_alpha1)) { if (indexVersionCreated.onOrAfter(Version.V_5_0_0_alpha1)) {
MappedFieldType fieldType = context.fieldMapper(field); MappedFieldType fieldType = context.fieldMapper(field);
if (fieldType == null) { if (fieldType == null) {
@ -418,11 +424,11 @@ public class PercolateQueryBuilder extends AbstractQueryBuilder<PercolateQueryBu
"] to be of type [percolator], but is of type [" + fieldType.typeName() + "]"); "] to be of type [percolator], but is of type [" + fieldType.typeName() + "]");
} }
PercolatorFieldMapper.FieldType pft = (PercolatorFieldMapper.FieldType) fieldType; PercolatorFieldMapper.FieldType pft = (PercolatorFieldMapper.FieldType) fieldType;
PercolateQuery.QueryStore queryStore = createStore(pft, context, mapUnmappedFieldsAsString); PercolateQuery.QueryStore queryStore = createStore(pft, percolateShardContext, mapUnmappedFieldsAsString);
return pft.percolateQuery(documentType, queryStore, document, docSearcher); return pft.percolateQuery(documentType, queryStore, document, docSearcher);
} else { } else {
Query percolateTypeQuery = new TermQuery(new Term(TypeFieldMapper.NAME, MapperService.PERCOLATOR_LEGACY_TYPE_NAME)); Query percolateTypeQuery = new TermQuery(new Term(TypeFieldMapper.NAME, MapperService.PERCOLATOR_LEGACY_TYPE_NAME));
PercolateQuery.QueryStore queryStore = createLegacyStore(context, mapUnmappedFieldsAsString); PercolateQuery.QueryStore queryStore = createLegacyStore(percolateShardContext, mapUnmappedFieldsAsString);
return new PercolateQuery(documentType, queryStore, document, percolateTypeQuery, docSearcher, return new PercolateQuery(documentType, queryStore, document, percolateTypeQuery, docSearcher,
new MatchNoDocsQuery("pre 5.0.0-alpha1 index, no verified matches")); new MatchNoDocsQuery("pre 5.0.0-alpha1 index, no verified matches"));
} }

View File

@ -248,4 +248,9 @@ public class PercolateQueryBuilderTests extends AbstractQueryTestCase<PercolateQ
throw new RuntimeException(e); throw new RuntimeException(e);
} }
} }
@Override
protected boolean isCachable(PercolateQueryBuilder queryBuilder) {
return false;
}
} }

View File

@ -42,6 +42,9 @@ import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.query.QueryShardException; import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.index.query.functionscore.WeightBuilder; import org.elasticsearch.index.query.functionscore.WeightBuilder;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.MockScriptPlugin;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder; import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
@ -59,13 +62,13 @@ import java.util.Map;
import java.util.NavigableSet; import java.util.NavigableSet;
import java.util.Set; import java.util.Set;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.function.Function;
import static org.elasticsearch.percolator.PercolateSourceBuilder.docBuilder; import static org.elasticsearch.percolator.PercolateSourceBuilder.docBuilder;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.common.xcontent.XContentFactory.smileBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.smileBuilder;
import static org.elasticsearch.common.xcontent.XContentFactory.yamlBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.yamlBuilder;
import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery; import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery;
import static org.elasticsearch.index.query.QueryBuilders.geoShapeQuery; import static org.elasticsearch.index.query.QueryBuilders.geoShapeQuery;
import static org.elasticsearch.index.query.QueryBuilders.hasChildQuery; import static org.elasticsearch.index.query.QueryBuilders.hasChildQuery;
@ -99,6 +102,7 @@ public class PercolatorIT extends ESIntegTestCase {
return Collections.singleton(PercolatorPlugin.class); return Collections.singleton(PercolatorPlugin.class);
} }
@Override @Override
protected Collection<Class<? extends Plugin>> transportClientPlugins() { protected Collection<Class<? extends Plugin>> transportClientPlugins() {
return Collections.singleton(PercolatorPlugin.class); return Collections.singleton(PercolatorPlugin.class);

View File

@ -21,6 +21,7 @@ package org.elasticsearch.percolator;
import org.apache.lucene.search.join.ScoreMode; import org.apache.lucene.search.join.ScoreMode;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
@ -31,12 +32,20 @@ import org.elasticsearch.index.query.MultiMatchQueryBuilder;
import org.elasticsearch.index.query.Operator; import org.elasticsearch.index.query.Operator;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.MockScriptPlugin;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder; import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.ESSingleNodeTestCase;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.function.Function;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
@ -49,7 +58,13 @@ import static org.elasticsearch.index.query.QueryBuilders.spanNearQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanNotQuery; import static org.elasticsearch.index.query.QueryBuilders.spanNotQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanTermQuery; import static org.elasticsearch.index.query.QueryBuilders.spanTermQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery; import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.percolator.PercolateSourceBuilder.docBuilder;
import static org.elasticsearch.percolator.PercolatorTestUtil.assertMatchCount;
import static org.elasticsearch.percolator.PercolatorTestUtil.convertFromTextArray;
import static org.elasticsearch.percolator.PercolatorTestUtil.preparePercolate;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.arrayContainingInAnyOrder;
import static org.hamcrest.Matchers.arrayWithSize;
import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.instanceOf;
@ -58,7 +73,33 @@ public class PercolatorQuerySearchIT extends ESSingleNodeTestCase {
@Override @Override
protected Collection<Class<? extends Plugin>> getPlugins() { protected Collection<Class<? extends Plugin>> getPlugins() {
return Collections.singleton(PercolatorPlugin.class); return Arrays.asList(PercolatorPlugin.class, CustomScriptPlugin.class);
}
public static class CustomScriptPlugin extends MockScriptPlugin {
@Override
protected Map<String, Function<Map<String, Object>, Object>> pluginScripts() {
Map<String, Function<Map<String, Object>, Object>> scripts = new HashMap<>();
scripts.put("1==1", vars -> Boolean.TRUE);
return scripts;
}
}
public void testPercolateScriptQuery() throws IOException {
client().admin().indices().prepareCreate("index").addMapping("type", "query", "type=percolator").get();
ensureGreen();
client().prepareIndex("index", "type", "1")
.setSource(jsonBuilder().startObject().field("query", QueryBuilders.scriptQuery(
new Script("1==1", ScriptService.ScriptType.INLINE, CustomScriptPlugin.NAME, null))).endObject())
.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.execute().actionGet();
PercolateResponse response = preparePercolate(client())
.setIndices("index").setDocumentType("type")
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", "b").endObject()))
.execute().actionGet();
assertMatchCount(response, 1L);
assertThat(response.getMatches(), arrayWithSize(1));
assertThat(convertFromTextArray(response.getMatches(), "index"), arrayContainingInAnyOrder("1"));
} }
public void testPercolatorQuery() throws Exception { public void testPercolatorQuery() throws Exception {

View File

@ -580,11 +580,23 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
public void testToQuery() throws IOException { public void testToQuery() throws IOException {
for (int runs = 0; runs < NUMBER_OF_TESTQUERIES; runs++) { for (int runs = 0; runs < NUMBER_OF_TESTQUERIES; runs++) {
QueryShardContext context = createShardContext(); QueryShardContext context = createShardContext();
assert context.isCachable();
context.setAllowUnmappedFields(true); context.setAllowUnmappedFields(true);
QB firstQuery = createTestQueryBuilder(); QB firstQuery = createTestQueryBuilder();
QB controlQuery = copyQuery(firstQuery); QB controlQuery = copyQuery(firstQuery);
setSearchContext(randomTypes, context); // only set search context for toQuery to be more realistic setSearchContext(randomTypes, context); // only set search context for toQuery to be more realistic
Query firstLuceneQuery = rewriteQuery(firstQuery, context).toQuery(context); /* we use a private rewrite context here since we want the most realistic way of asserting that we are cachabel or not.
* We do it this way in SearchService where
* we first rewrite the query with a private context, then reset the context and then build the actual lucene query*/
QueryBuilder rewritten = rewriteQuery(firstQuery, new QueryShardContext(context));
Query firstLuceneQuery = rewritten.toQuery(context);
if (isCachable(firstQuery)) {
assertTrue("query was marked as not cacheable in the context but this test indicates it should be cacheable: "
+ firstQuery.toString(), context.isCachable());
} else {
assertFalse("query was marked as cacheable in the context but this test indicates it should not be cacheable: "
+ firstQuery.toString(), context.isCachable());
}
assertNotNull("toQuery should not return null", firstLuceneQuery); assertNotNull("toQuery should not return null", firstLuceneQuery);
assertLuceneQuery(firstQuery, firstLuceneQuery, context); assertLuceneQuery(firstQuery, firstLuceneQuery, context);
//remove after assertLuceneQuery since the assertLuceneQuery impl might access the context as well //remove after assertLuceneQuery since the assertLuceneQuery impl might access the context as well
@ -636,6 +648,10 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
return rewritten; return rewritten;
} }
protected boolean isCachable(QB queryBuilder) {
return true;
}
/** /**
* Few queries allow you to set the boost and queryName on the java api, although the corresponding parser * Few queries allow you to set the boost and queryName on the java api, although the corresponding parser
* doesn't parse them as they are not supported. This method allows to disable boost and queryName related tests for those queries. * doesn't parse them as they are not supported. This method allows to disable boost and queryName related tests for those queries.
@ -1020,6 +1036,7 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
private final BitsetFilterCache bitsetFilterCache; private final BitsetFilterCache bitsetFilterCache;
private final ScriptService scriptService; private final ScriptService scriptService;
private final Client client; private final Client client;
private final long nowInMillis = randomPositiveLong();
ServiceHolder(Settings nodeSettings, Settings indexSettings, ServiceHolder(Settings nodeSettings, Settings indexSettings,
Collection<Class<? extends Plugin>> plugins, AbstractQueryTestCase<?> testCase) throws IOException { Collection<Class<? extends Plugin>> plugins, AbstractQueryTestCase<?> testCase) throws IOException {
@ -1098,7 +1115,7 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
QueryShardContext createShardContext() { QueryShardContext createShardContext() {
ClusterState state = ClusterState.builder(new ClusterName("_name")).build(); ClusterState state = ClusterState.builder(new ClusterName("_name")).build();
return new QueryShardContext(idxSettings, bitsetFilterCache, indexFieldDataService, mapperService, similarityService, return new QueryShardContext(idxSettings, bitsetFilterCache, indexFieldDataService, mapperService, similarityService,
scriptService, indicesQueriesRegistry, this.client, null, state); scriptService, indicesQueriesRegistry, this.client, null, state, () -> nowInMillis);
} }
ScriptModule createScriptModule(List<ScriptPlugin> scriptPlugins) { ScriptModule createScriptModule(List<ScriptPlugin> scriptPlugins) {

View File

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

View File

@ -33,8 +33,9 @@ import org.elasticsearch.test.TestSearchContext;
public class MockSearchServiceTests extends ESTestCase { public class MockSearchServiceTests extends ESTestCase {
public void testAssertNoInFlightContext() { public void testAssertNoInFlightContext() {
final long nowInMillis = randomPositiveLong();
SearchContext s = new TestSearchContext(new QueryShardContext(new IndexSettings(IndexMetaData.PROTO, Settings.EMPTY), null, null, 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, () -> nowInMillis)) {
@Override @Override
public SearchShardTarget shardTarget() { public SearchShardTarget shardTarget() {
return new SearchShardTarget("node", new Index("idx", "ignored"), 0); return new SearchShardTarget("node", new Index("idx", "ignored"), 0);