diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java index 442b0915e3b..52d45ec9407 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java @@ -473,6 +473,14 @@ public class SearchRequestBuilder extends ActionRequestBuilderfalse + */ + public SearchRequestBuilder setProfile(boolean profile) { + sourceBuilder().profile(profile); + return this; + } + @Override public String toString() { if (request.source() != null) { diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java b/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java index 769e0978a71..e6681bf2b9f 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.search; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.unit.TimeValue; @@ -32,9 +33,12 @@ import org.elasticsearch.rest.action.support.RestActions; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.internal.InternalSearchResponse; +import org.elasticsearch.search.profile.ProfileShardResult; import org.elasticsearch.search.suggest.Suggest; import java.io.IOException; +import java.util.List; +import java.util.Map; import static org.elasticsearch.action.search.ShardSearchFailure.readShardSearchFailure; import static org.elasticsearch.search.internal.InternalSearchResponse.readInternalSearchResponse; @@ -160,6 +164,16 @@ public class SearchResponse extends ActionResponse implements StatusToXContent { this.scrollId = scrollId; } + /** + * If profiling was enabled, this returns an object containing the profile results from + * each shard. If profiling was not enabled, this will return null + * + * @return The profile results or null + */ + public @Nullable Map> getProfileResults() { + return internalResponse.profile(); + } + static final class Fields { static final XContentBuilderString _SCROLL_ID = new XContentBuilderString("_scroll_id"); static final XContentBuilderString TOOK = new XContentBuilderString("took"); diff --git a/core/src/main/java/org/elasticsearch/percolator/PercolateContext.java b/core/src/main/java/org/elasticsearch/percolator/PercolateContext.java index 5b09b55f8dc..8df956f2cea 100644 --- a/core/src/main/java/org/elasticsearch/percolator/PercolateContext.java +++ b/core/src/main/java/org/elasticsearch/percolator/PercolateContext.java @@ -43,7 +43,6 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; -import org.elasticsearch.index.cache.query.QueryCache; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MappedFieldType; @@ -74,6 +73,8 @@ import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.lookup.LeafSearchLookup; import org.elasticsearch.search.lookup.SearchLookup; +import org.elasticsearch.search.profile.Profiler; +import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rescore.RescoreSearchContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; @@ -139,7 +140,7 @@ public class PercolateContext extends SearchContext { this.bigArrays = bigArrays.withCircuitBreaking(); this.querySearchResult = new QuerySearchResult(0, searchShardTarget); this.engineSearcher = indexShard.acquireSearcher("percolate"); - this.searcher = new ContextIndexSearcher(this, engineSearcher); + this.searcher = new ContextIndexSearcher(engineSearcher, indexService.cache().query(), indexShard.getQueryCachingPolicy()); this.scriptService = scriptService; this.numberOfShards = request.getNumberOfShards(); this.aliasFilter = aliasFilter; @@ -748,5 +749,7 @@ public class PercolateContext extends SearchContext { } @Override - public QueryCache getQueryCache() { return indexService.cache().query();} + public Profilers getProfilers() { + throw new UnsupportedOperationException(); + } } diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 00939a74cf8..29ad7ccf0a1 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -23,6 +23,7 @@ import com.carrotsearch.hppc.ObjectFloatHashMap; import com.carrotsearch.hppc.ObjectHashSet; import com.carrotsearch.hppc.ObjectSet; import com.carrotsearch.hppc.cursors.ObjectCursor; + import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; @@ -86,6 +87,7 @@ import org.elasticsearch.search.fetch.script.ScriptFieldsContext.ScriptField; import org.elasticsearch.search.highlight.HighlightBuilder; import org.elasticsearch.search.internal.*; import org.elasticsearch.search.internal.SearchContext.Lifetime; +import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.*; import org.elasticsearch.search.warmer.IndexWarmersMetaData; import org.elasticsearch.threadpool.ThreadPool; @@ -552,7 +554,7 @@ public class SearchService extends AbstractLifecycleComponent imp Engine.Searcher engineSearcher = searcher == null ? indexShard.acquireSearcher("search") : searcher; - SearchContext context = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, engineSearcher, indexService, indexShard, scriptService, pageCacheRecycler, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher, defaultSearchTimeout); + DefaultSearchContext context = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, engineSearcher, indexService, indexShard, scriptService, pageCacheRecycler, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher, defaultSearchTimeout); SearchContext.setCurrent(context); try { @@ -659,7 +661,7 @@ public class SearchService extends AbstractLifecycleComponent imp } } - private void parseSource(SearchContext context, SearchSourceBuilder source) throws SearchContextException { + private void parseSource(DefaultSearchContext context, SearchSourceBuilder source) throws SearchContextException { // nothing to parse... if (source == null) { return; @@ -715,6 +717,9 @@ public class SearchService extends AbstractLifecycleComponent imp if (source.minScore() != null) { context.minimumScore(source.minScore()); } + if (source.profile()) { + context.setProfilers(new Profilers(context.searcher())); + } context.timeoutInMillis(source.timeoutInMillis()); context.terminateAfter(source.terminateAfter()); if (source.aggregations() != null) { diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java b/core/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java index 742f678f6ff..0681996e3ec 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java @@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations; import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.Collector; import org.apache.lucene.search.Query; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lucene.search.Queries; @@ -30,10 +31,13 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.SiblingPipelineAggregator; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.profile.CollectorResult; +import org.elasticsearch.search.profile.InternalProfileCollector; import org.elasticsearch.search.query.QueryPhaseExecutionException; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -81,8 +85,13 @@ public class AggregationPhase implements SearchPhase { } context.aggregations().aggregators(aggregators); if (!collectors.isEmpty()) { - final BucketCollector collector = BucketCollector.wrap(collectors); - collector.preCollection(); + Collector collector = BucketCollector.wrap(collectors); + ((BucketCollector)collector).preCollection(); + if (context.getProfilers() != null) { + collector = new InternalProfileCollector(collector, CollectorResult.REASON_AGGREGATION, + // TODO: report on child aggs as well + Collections.emptyList()); + } context.queryCollectors().put(AggregationPhase.class, collector); } } catch (IOException e) { @@ -116,6 +125,7 @@ public class AggregationPhase implements SearchPhase { BucketCollector globalsCollector = BucketCollector.wrap(globals); Query query = Queries.newMatchAllQuery(); Query searchFilter = context.searchFilter(context.types()); + if (searchFilter != null) { BooleanQuery filtered = new BooleanQuery.Builder() .add(query, Occur.MUST) @@ -124,8 +134,20 @@ public class AggregationPhase implements SearchPhase { query = filtered; } try { + final Collector collector; + if (context.getProfilers() == null) { + collector = globalsCollector; + } else { + InternalProfileCollector profileCollector = new InternalProfileCollector( + globalsCollector, CollectorResult.REASON_AGGREGATION_GLOBAL, + // TODO: report on sub collectors + Collections.emptyList()); + collector = profileCollector; + // start a new profile with this collector + context.getProfilers().addProfiler().setCollector(profileCollector); + } globalsCollector.preCollection(); - context.searcher().search(query, globalsCollector); + context.searcher().search(query, collector); } catch (Exception e) { throw new QueryPhaseExecutionException(context, "Failed to execute global aggregators", e); } finally { diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java b/core/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java index ee38e2b3610..c1c1bff1adb 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java @@ -25,6 +25,7 @@ import org.apache.lucene.search.Collector; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.stream.StreamSupport; @@ -99,6 +100,11 @@ public abstract class BucketCollector implements Collector { } return false; } + + @Override + public String toString() { + return Arrays.toString(collectors); + } }; } } diff --git a/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java b/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java index 465729ca714..3ea2d604b8a 100644 --- a/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java @@ -22,6 +22,7 @@ package org.elasticsearch.search.builder; import com.carrotsearch.hppc.ObjectFloatHashMap; import com.carrotsearch.hppc.cursors.ObjectCursor; +import org.elasticsearch.Version; import org.elasticsearch.action.support.ToXContentToBytes; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; @@ -91,6 +92,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ public static final ParseField RESCORE_FIELD = new ParseField("rescore"); public static final ParseField STATS_FIELD = new ParseField("stats"); public static final ParseField EXT_FIELD = new ParseField("ext"); + public static final ParseField PROFILE_FIELD = new ParseField("profile"); private static final SearchSourceBuilder PROTOTYPE = new SearchSourceBuilder(); @@ -158,6 +160,9 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ private BytesReference ext = null; + private boolean profile = false; + + /** * Constructs a new search source builder. */ @@ -475,6 +480,22 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ return this; } + /** + * Should the query be profiled. Defaults to false + */ + public SearchSourceBuilder profile(boolean profile) { + this.profile = profile; + return this; + } + + /** + * Return whether to profile query execution, or {@code null} if + * unspecified. + */ + public boolean profile() { + return profile; + } + /** * Gets the bytes representing the rescore builders for this request. */ @@ -723,6 +744,8 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ builder.fieldNames = fieldNames; } else if (context.parseFieldMatcher().match(currentFieldName, SORT_FIELD)) { builder.sort(parser.text()); + } else if (context.parseFieldMatcher().match(currentFieldName, PROFILE_FIELD)) { + builder.profile = parser.booleanValue(); } else { throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].", parser.getTokenLocation()); @@ -931,6 +954,10 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ builder.field(EXPLAIN_FIELD.getPreferredName(), explain); } + if (profile) { + builder.field("profile", true); + } + if (fetchSourceContext != null) { builder.field(_SOURCE_FIELD.getPreferredName(), fetchSourceContext); } @@ -1212,6 +1239,11 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ if (in.readBoolean()) { builder.ext = in.readBytesReference(); } + if (in.getVersion().onOrAfter(Version.V_2_2_0)) { + builder.profile = in.readBoolean(); + } else { + builder.profile = false; + } return builder; } @@ -1325,13 +1357,16 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ if (hasExt) { out.writeBytesReference(ext); } + if (out.getVersion().onOrAfter(Version.V_2_2_0)) { + out.writeBoolean(profile); + } } @Override public int hashCode() { return Objects.hash(aggregations, explain, fetchSourceContext, fieldDataFields, fieldNames, from, highlightBuilder, indexBoost, innerHitsBuilder, minScore, postQueryBuilder, queryBuilder, rescoreBuilders, scriptFields, - size, sorts, stats, suggestBuilder, terminateAfter, timeoutInMillis, trackScores, version); + size, sorts, stats, suggestBuilder, terminateAfter, timeoutInMillis, trackScores, version, profile); } @Override @@ -1364,6 +1399,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ && Objects.equals(terminateAfter, other.terminateAfter) && Objects.equals(timeoutInMillis, other.timeoutInMillis) && Objects.equals(trackScores, other.trackScores) - && Objects.equals(version, other.version); + && Objects.equals(version, other.version) + && Objects.equals(profile, other.profile); } } diff --git a/core/src/main/java/org/elasticsearch/search/controller/SearchPhaseController.java b/core/src/main/java/org/elasticsearch/search/controller/SearchPhaseController.java index f76527163cb..835e6e71425 100644 --- a/core/src/main/java/org/elasticsearch/search/controller/SearchPhaseController.java +++ b/core/src/main/java/org/elasticsearch/search/controller/SearchPhaseController.java @@ -43,7 +43,6 @@ import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.SiblingPipelineAggregator; import org.elasticsearch.search.dfs.AggregatedDfs; import org.elasticsearch.search.dfs.DfsSearchResult; @@ -52,9 +51,11 @@ import org.elasticsearch.search.fetch.FetchSearchResultProvider; import org.elasticsearch.search.internal.InternalSearchHit; import org.elasticsearch.search.internal.InternalSearchHits; import org.elasticsearch.search.internal.InternalSearchResponse; +import org.elasticsearch.search.profile.InternalProfileShardResults; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.query.QuerySearchResultProvider; import org.elasticsearch.search.suggest.Suggest; +import org.elasticsearch.search.profile.ProfileShardResult; import java.io.IOException; import java.util.ArrayList; @@ -410,6 +411,17 @@ public class SearchPhaseController extends AbstractComponent { } } + //Collect profile results + InternalProfileShardResults shardResults = null; + if (!queryResults.isEmpty() && firstResult.profileResults() != null) { + Map> profileResults = new HashMap<>(queryResults.size()); + for (AtomicArray.Entry entry : queryResults) { + String key = entry.value.queryResult().shardTarget().toString(); + profileResults.put(key, entry.value.queryResult().profileResults()); + } + shardResults = new InternalProfileShardResults(profileResults); + } + if (aggregations != null) { List pipelineAggregators = firstResult.pipelineAggregators(); if (pipelineAggregators != null) { @@ -427,7 +439,7 @@ public class SearchPhaseController extends AbstractComponent { InternalSearchHits searchHits = new InternalSearchHits(hits.toArray(new InternalSearchHit[hits.size()]), totalHits, maxScore); - return new InternalSearchResponse(searchHits, aggregations, suggest, timedOut, terminatedEarly); + return new InternalSearchResponse(searchHits, aggregations, suggest, shardResults, timedOut, terminatedEarly); } } diff --git a/core/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java b/core/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java index 0a9b860edb7..a7bacb64d94 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java +++ b/core/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java @@ -26,6 +26,9 @@ import org.apache.lucene.search.*; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.search.dfs.AggregatedDfs; +import org.elasticsearch.search.profile.ProfileBreakdown; +import org.elasticsearch.search.profile.ProfileWeight; +import org.elasticsearch.search.profile.Profiler; import java.io.IOException; @@ -43,26 +46,44 @@ public class ContextIndexSearcher extends IndexSearcher implements Releasable { private final Engine.Searcher engineSearcher; - public ContextIndexSearcher(SearchContext searchContext, Engine.Searcher searcher) { + // TODO revisit moving the profiler to inheritance or wrapping model in the future + private Profiler profiler; + + public ContextIndexSearcher(Engine.Searcher searcher, + QueryCache queryCache, QueryCachingPolicy queryCachingPolicy) { super(searcher.reader()); in = searcher.searcher(); engineSearcher = searcher; setSimilarity(searcher.searcher().getSimilarity(true)); - setQueryCache(searchContext.getQueryCache()); - setQueryCachingPolicy(searchContext.indexShard().getQueryCachingPolicy()); + setQueryCache(queryCache); + setQueryCachingPolicy(queryCachingPolicy); } @Override public void close() { } + public void setProfiler(Profiler profiler) { + this.profiler = profiler; + } + public void setAggregatedDfs(AggregatedDfs aggregatedDfs) { this.aggregatedDfs = aggregatedDfs; } @Override public Query rewrite(Query original) throws IOException { - return in.rewrite(original); + if (profiler != null) { + profiler.startRewriteTime(); + } + + try { + return in.rewrite(original); + } finally { + if (profiler != null) { + profiler.stopAndAddRewriteTime(); + } + } } @Override @@ -72,8 +93,34 @@ public class ContextIndexSearcher extends IndexSearcher implements Releasable { if (aggregatedDfs != null && needsScores) { // if scores are needed and we have dfs data then use it return super.createNormalizedWeight(query, needsScores); + } else if (profiler != null) { + // we need to use the createWeight method to insert the wrappers + return super.createNormalizedWeight(query, needsScores); + } else { + return in.createNormalizedWeight(query, needsScores); + } + } + + @Override + public Weight createWeight(Query query, boolean needsScores) throws IOException { + if (profiler != null) { + // createWeight() is called for each query in the tree, so we tell the queryProfiler + // each invocation so that it can build an internal representation of the query + // tree + ProfileBreakdown profile = profiler.getQueryBreakdown(query); + profile.startTime(ProfileBreakdown.TimingType.CREATE_WEIGHT); + final Weight weight; + try { + weight = super.createWeight(query, needsScores); + } finally { + profile.stopAndRecordTime(); + profiler.pollLastQuery(); + } + return new ProfileWeight(query, weight, profile); + } else { + // needs to be 'super', not 'in' in order to use aggregated DFS + return super.createWeight(query, needsScores); } - return in.createNormalizedWeight(query, needsScores); } @Override diff --git a/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java b/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java index 1174fcdd8a9..2d3f6590629 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java +++ b/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java @@ -58,6 +58,8 @@ import org.elasticsearch.search.fetch.source.FetchSourceContext; import org.elasticsearch.search.highlight.SearchContextHighlight; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.query.QueryPhaseExecutionException; +import org.elasticsearch.search.profile.Profiler; +import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rescore.RescoreSearchContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; @@ -129,10 +131,10 @@ public class DefaultSearchContext extends SearchContext { private List rescore; private SearchLookup searchLookup; private volatile long keepAlive; - private ScoreDoc lastEmittedDoc; private final long originNanoTime = System.nanoTime(); private volatile long lastAccessTime = -1; private InnerHitsContext innerHitsContext; + private Profilers profilers; private final Map subPhaseContexts = new HashMap<>(); private final Map, Collector> queryCollectors = new HashMap<>(); @@ -158,7 +160,7 @@ public class DefaultSearchContext extends SearchContext { this.fetchResult = new FetchSearchResult(id, shardTarget); this.indexShard = indexShard; this.indexService = indexService; - this.searcher = new ContextIndexSearcher(this, engineSearcher); + this.searcher = new ContextIndexSearcher(engineSearcher, indexService.cache().query(), indexShard.getQueryCachingPolicy()); this.timeEstimateCounter = timeEstimateCounter; this.timeoutInMillis = timeout.millis(); } @@ -724,5 +726,11 @@ public class DefaultSearchContext extends SearchContext { } @Override - public QueryCache getQueryCache() { return indexService.cache().query();} + public Profilers getProfilers() { + return profilers; + } + + public void setProfilers(Profilers profilers) { + this.profilers = profilers; + } } diff --git a/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java b/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java index 7225c7b32bd..1f04d013401 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java +++ b/core/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java @@ -29,7 +29,6 @@ import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; -import org.elasticsearch.index.cache.query.QueryCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; @@ -49,6 +48,7 @@ import org.elasticsearch.search.fetch.script.ScriptFieldsContext; import org.elasticsearch.search.fetch.source.FetchSourceContext; import org.elasticsearch.search.highlight.SearchContextHighlight; import org.elasticsearch.search.lookup.SearchLookup; +import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rescore.RescoreSearchContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; @@ -517,8 +517,11 @@ public abstract class FilteredSearchContext extends SearchContext { } @Override - public Map, Collector> queryCollectors() { return in.queryCollectors();} + public Profilers getProfilers() { + return in.getProfilers(); + } @Override - public QueryCache getQueryCache() { return in.getQueryCache();} + public Map, Collector> queryCollectors() { return in.queryCollectors();} + } diff --git a/core/src/main/java/org/elasticsearch/search/internal/InternalSearchResponse.java b/core/src/main/java/org/elasticsearch/search/internal/InternalSearchResponse.java index 7b73772f9da..b8255e0bb52 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/InternalSearchResponse.java +++ b/core/src/main/java/org/elasticsearch/search/internal/InternalSearchResponse.java @@ -28,9 +28,14 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.profile.InternalProfileShardResults; +import org.elasticsearch.search.profile.ProfileShardResult; import org.elasticsearch.search.suggest.Suggest; import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; import static org.elasticsearch.search.internal.InternalSearchHits.readSearchHits; @@ -40,7 +45,7 @@ import static org.elasticsearch.search.internal.InternalSearchHits.readSearchHit public class InternalSearchResponse implements Streamable, ToXContent { public static InternalSearchResponse empty() { - return new InternalSearchResponse(InternalSearchHits.empty(), null, null, false, null); + return new InternalSearchResponse(InternalSearchHits.empty(), null, null, null, false, null); } private InternalSearchHits hits; @@ -49,6 +54,8 @@ public class InternalSearchResponse implements Streamable, ToXContent { private Suggest suggest; + private InternalProfileShardResults profileResults; + private boolean timedOut; private Boolean terminatedEarly = null; @@ -56,10 +63,12 @@ public class InternalSearchResponse implements Streamable, ToXContent { private InternalSearchResponse() { } - public InternalSearchResponse(InternalSearchHits hits, InternalAggregations aggregations, Suggest suggest, boolean timedOut, Boolean terminatedEarly) { + public InternalSearchResponse(InternalSearchHits hits, InternalAggregations aggregations, Suggest suggest, + InternalProfileShardResults profileResults, boolean timedOut, Boolean terminatedEarly) { this.hits = hits; this.aggregations = aggregations; this.suggest = suggest; + this.profileResults = profileResults; this.timedOut = timedOut; this.terminatedEarly = terminatedEarly; } @@ -84,6 +93,19 @@ public class InternalSearchResponse implements Streamable, ToXContent { return suggest; } + /** + * Returns the profile results for this search response (including all shards). + * An empty map is returned if profiling was not enabled + * + * @return Profile results + */ + public Map> profile() { + if (profileResults == null) { + return Collections.emptyMap(); + } + return profileResults.getShardResults(); + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { hits.toXContent(builder, params); @@ -93,6 +115,9 @@ public class InternalSearchResponse implements Streamable, ToXContent { if (suggest != null) { suggest.toXContent(builder, params); } + if (profileResults != null) { + profileResults.toXContent(builder, params); + } return builder; } @@ -114,6 +139,12 @@ public class InternalSearchResponse implements Streamable, ToXContent { timedOut = in.readBoolean(); terminatedEarly = in.readOptionalBoolean(); + + if (in.getVersion().onOrAfter(Version.V_2_2_0) && in.readBoolean()) { + profileResults = new InternalProfileShardResults(in); + } else { + profileResults = null; + } } @Override @@ -134,5 +165,14 @@ public class InternalSearchResponse implements Streamable, ToXContent { out.writeBoolean(timedOut); out.writeOptionalBoolean(terminatedEarly); + + if (out.getVersion().onOrAfter(Version.V_2_2_0)) { + if (profileResults == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + profileResults.writeTo(out); + } + } } } diff --git a/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java b/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java index 0f61b2bc6a3..4e4e9dd5dd7 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/core/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -35,7 +35,6 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; -import org.elasticsearch.index.cache.query.QueryCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; @@ -56,6 +55,7 @@ import org.elasticsearch.search.fetch.script.ScriptFieldsContext; import org.elasticsearch.search.fetch.source.FetchSourceContext; import org.elasticsearch.search.highlight.SearchContextHighlight; import org.elasticsearch.search.lookup.SearchLookup; +import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rescore.RescoreSearchContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; @@ -303,6 +303,11 @@ public abstract class SearchContext extends DelegatingHasContextAndHeaders imple public abstract FetchSearchResult fetchResult(); + /** + * Return a handle over the profilers for the current search request, or {@code null} if profiling is not enabled. + */ + public abstract Profilers getProfilers(); + /** * Schedule the release of a resource. The time when {@link Releasable#close()} will be called on this object * is function of the provided {@link Lifetime}. @@ -367,5 +372,4 @@ public abstract class SearchContext extends DelegatingHasContextAndHeaders imple CONTEXT } - public abstract QueryCache getQueryCache(); } diff --git a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java index 47791aeddfa..9d15dfd5790 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java +++ b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java @@ -71,6 +71,8 @@ public class ShardSearchLocalRequest extends ContextAndHeaderHolder implements S private Boolean requestCache; private long nowInMillis; + private boolean profile; + ShardSearchLocalRequest() { } @@ -165,6 +167,16 @@ public class ShardSearchLocalRequest extends ContextAndHeaderHolder implements S return scroll; } + @Override + public void setProfile(boolean profile) { + this.profile = profile; + } + + @Override + public boolean isProfile() { + return profile; + } + @SuppressWarnings("unchecked") protected void innerReadFrom(StreamInput in) throws IOException { index = in.readString(); diff --git a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index fb631b08270..b1730b6a14e 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -59,6 +59,17 @@ public interface ShardSearchRequest extends HasContextAndHeaders { Scroll scroll(); + /** + * Sets if this shard search needs to be profiled or not + * @param profile True if the shard should be profiled + */ + void setProfile(boolean profile); + + /** + * Returns true if this shard search is being profiled or not + */ + boolean isProfile(); + /** * Returns the cache key for this shard search request, based on its content */ diff --git a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java index 279d9d6bd20..0f9c0ced411 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java +++ b/core/src/main/java/org/elasticsearch/search/internal/ShardSearchTransportRequest.java @@ -150,4 +150,14 @@ public class ShardSearchTransportRequest extends TransportRequest implements Sha public BytesReference cacheKey() throws IOException { return shardSearchLocalRequest.cacheKey(); } + + @Override + public void setProfile(boolean profile) { + shardSearchLocalRequest.setProfile(profile); + } + + @Override + public boolean isProfile() { + return shardSearchLocalRequest.isProfile(); + } } diff --git a/core/src/main/java/org/elasticsearch/search/profile/CollectorResult.java b/core/src/main/java/org/elasticsearch/search/profile/CollectorResult.java new file mode 100644 index 00000000000..4949c6388d2 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/CollectorResult.java @@ -0,0 +1,156 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; + +/** + * Public interface and serialization container for profiled timings of the + * Collectors used in the search. Children CollectorResult's may be + * embedded inside of a parent CollectorResult + */ +public class CollectorResult implements ToXContent, Writeable { + + public static final String REASON_SEARCH_COUNT = "search_count"; + public static final String REASON_SEARCH_TOP_HITS = "search_top_hits"; + public static final String REASON_SEARCH_TERMINATE_AFTER_COUNT = "search_terminate_after_count"; + public static final String REASON_SEARCH_POST_FILTER = "search_post_filter"; + public static final String REASON_SEARCH_MIN_SCORE = "search_min_score"; + public static final String REASON_SEARCH_MULTI = "search_multi"; + public static final String REASON_SEARCH_TIMEOUT = "search_timeout"; + public static final String REASON_AGGREGATION = "aggregation"; + public static final String REASON_AGGREGATION_GLOBAL = "aggregation_global"; + + private static final ParseField NAME = new ParseField("name"); + private static final ParseField REASON = new ParseField("reason"); + private static final ParseField TIME = new ParseField("time"); + private static final ParseField CHILDREN = new ParseField("children"); + + /** + * A more friendly representation of the Collector's class name + */ + private final String collectorName; + + /** + * A "hint" to help provide some context about this Collector + */ + private final String reason; + + /** + * The total elapsed time for this Collector + */ + private final Long time; + + /** + * A list of children collectors "embedded" inside this collector + */ + private List children; + + public CollectorResult(String collectorName, String reason, Long time, List children) { + this.collectorName = collectorName; + this.reason = reason; + this.time = time; + this.children = children; + } + + public CollectorResult(StreamInput in) throws IOException { + this.collectorName = in.readString(); + this.reason = in.readString(); + this.time = in.readLong(); + int size = in.readVInt(); + this.children = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + CollectorResult child = new CollectorResult(in); + this.children.add(child); + } + } + + /** + * @return the profiled time for this collector (inclusive of children) + */ + public long getTime() { + return this.time; + } + + /** + * @return a human readable "hint" about what this collector was used for + */ + public String getReason() { + return this.reason; + } + + /** + * @return the lucene class name of the collector + */ + public String getName() { + return this.collectorName; + } + + /** + * @return a list of children collectors + */ + public List getProfiledChildren() { + return children; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder = builder.startObject() + .field(NAME.getPreferredName(), toString()) + .field(REASON.getPreferredName(), reason) + .field(TIME.getPreferredName(), String.format(Locale.US, "%.10gms", (double) (getTime() / 1000000.0))); + + if (!children.isEmpty()) { + builder = builder.startArray(CHILDREN.getPreferredName()); + for (CollectorResult child : children) { + builder = child.toXContent(builder, params); + } + builder = builder.endArray(); + } + builder = builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(collectorName); + out.writeString(reason); + out.writeLong(time); + out.writeVInt(children.size()); + for (CollectorResult child : children) { + child.writeTo(out); + } + } + + @Override + public Object readFrom(StreamInput in) throws IOException { + return new CollectorResult(in); + } +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/InternalProfileCollector.java b/core/src/main/java/org/elasticsearch/search/profile/InternalProfileCollector.java new file mode 100644 index 00000000000..132731f37c6 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/InternalProfileCollector.java @@ -0,0 +1,135 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.LeafCollector; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * This class wraps a Lucene Collector and times the execution of: + * - setScorer() + * - collect() + * - doSetNextReader() + * - needsScores() + * + * InternalProfiler facilitates the linking of the the Collector graph + */ +public class InternalProfileCollector implements Collector { + + /** + * A more friendly representation of the Collector's class name + */ + private final String collectorName; + + /** + * A "hint" to help provide some context about this Collector + */ + private final String reason; + + /** The wrapped collector */ + private final ProfileCollector collector; + + /** + * A list of "embedded" children collectors + */ + private final List children; + + public InternalProfileCollector(Collector collector, String reason, List children) { + this.collector = new ProfileCollector(collector); + this.reason = reason; + this.collectorName = deriveCollectorName(collector); + this.children = children; + } + + /** + * @return the profiled time for this collector (inclusive of children) + */ + public long getTime() { + return collector.getTime(); + } + + /** + * @return a human readable "hint" about what this collector was used for + */ + public String getReason() { + return this.reason; + } + + /** + * @return the lucene class name of the collector + */ + public String getName() { + return this.collectorName; + } + + /** + * Creates a human-friendly representation of the Collector name. + * + * Bucket Collectors use the aggregation name in their toString() method, + * which makes the profiled output a bit nicer. + * + * @param c The Collector to derive a name from + * @return A (hopefully) prettier name + */ + private String deriveCollectorName(Collector c) { + String s = c.getClass().getSimpleName(); + + // MutiCollector which wraps multiple BucketCollectors is generated + // via an anonymous class, so this corrects the lack of a name by + // asking the enclosingClass + if (s.equals("")) { + s = c.getClass().getEnclosingClass().getSimpleName(); + } + + // Aggregation collector toString()'s include the user-defined agg name + if (reason.equals(CollectorResult.REASON_AGGREGATION) || reason.equals(CollectorResult.REASON_AGGREGATION_GLOBAL)) { + s += ": [" + c.toString() + "]"; + } + return s; + } + + @Override + public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { + return collector.getLeafCollector(context); + } + + @Override + public boolean needsScores() { + return collector.needsScores(); + } + + public CollectorResult getCollectorTree() { + return InternalProfileCollector.doGetCollectorTree(this); + } + + private static CollectorResult doGetCollectorTree(InternalProfileCollector collector) { + List childResults = new ArrayList<>(collector.children.size()); + for (InternalProfileCollector child : collector.children) { + CollectorResult result = doGetCollectorTree(child); + childResults.add(result); + } + return new CollectorResult(collector.getName(), collector.getReason(), collector.getTime(), childResults); + } +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/InternalProfileShardResults.java b/core/src/main/java/org/elasticsearch/search/profile/InternalProfileShardResults.java new file mode 100644 index 00000000000..2ab3b632119 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/InternalProfileShardResults.java @@ -0,0 +1,89 @@ +package org.elasticsearch.search.profile; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.*; +import java.util.stream.Collectors; + +/** + * A container class to hold all the profile results across all shards. Internally + * holds a map of shard ID -> Profiled results + */ +public final class InternalProfileShardResults implements Writeable, ToXContent{ + + private Map> shardResults; + + public InternalProfileShardResults(Map> shardResults) { + Map> transformed = + shardResults.entrySet() + .stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + e -> Collections.unmodifiableList(e.getValue())) + ); + this.shardResults = Collections.unmodifiableMap(transformed); + } + + public InternalProfileShardResults(StreamInput in) throws IOException { + int size = in.readInt(); + shardResults = new HashMap<>(size); + + for (int i = 0; i < size; i++) { + String key = in.readString(); + int shardResultsSize = in.readInt(); + + List shardResult = new ArrayList<>(shardResultsSize); + + for (int j = 0; j < shardResultsSize; j++) { + ProfileShardResult result = new ProfileShardResult(in); + shardResult.add(result); + } + shardResults.put(key, shardResult); + } + } + + public Map> getShardResults() { + return this.shardResults; + } + + @Override + public InternalProfileShardResults readFrom(StreamInput in) throws IOException { + return new InternalProfileShardResults(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeInt(shardResults.size()); + for (Map.Entry> entry : shardResults.entrySet()) { + out.writeString(entry.getKey()); + out.writeInt(entry.getValue().size()); + + for (ProfileShardResult result : entry.getValue()) { + result.writeTo(out); + } + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject("profile").startArray("shards"); + + for (Map.Entry> entry : shardResults.entrySet()) { + builder.startObject().field("id",entry.getKey()).startArray("searches"); + for (ProfileShardResult result : entry.getValue()) { + builder.startObject(); + result.toXContent(builder, params); + builder.endObject(); + } + builder.endArray().endObject(); + } + + builder.endArray().endObject(); + return builder; + } +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/InternalProfileTree.java b/core/src/main/java/org/elasticsearch/search/profile/InternalProfileTree.java new file mode 100644 index 00000000000..4bc8a85a781 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/InternalProfileTree.java @@ -0,0 +1,235 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.apache.lucene.search.Query; + +import java.util.*; +import java.util.concurrent.LinkedBlockingDeque; + +/** + * This class tracks the dependency tree for queries (scoring and rewriting) and + * generates {@link ProfileBreakdown} for each node in the tree. It also finalizes the tree + * and returns a list of {@link ProfileResult} that can be serialized back to the client + */ +final class InternalProfileTree { + + private ArrayList timings; + + /** Maps the Query to it's list of children. This is basically the dependency tree */ + private ArrayList> tree; + + /** A list of the original queries, keyed by index position */ + private ArrayList queries; + + /** A list of top-level "roots". Each root can have its own tree of profiles */ + private ArrayList roots; + + /** Rewrite time */ + private long rewriteTime; + private long rewriteScratch; + + /** A temporary stack used to record where we are in the dependency tree. Only used by scoring queries */ + private Deque stack; + + private int currentToken = 0; + + public InternalProfileTree() { + timings = new ArrayList<>(10); + stack = new LinkedBlockingDeque<>(10); + tree = new ArrayList<>(10); + queries = new ArrayList<>(10); + roots = new ArrayList<>(10); + } + + /** + * Returns a {@link ProfileBreakdown} for a scoring query. Scoring queries (e.g. those + * that are past the rewrite phase and are now being wrapped by createWeight() ) follow + * a recursive progression. We can track the dependency tree by a simple stack + * + * The only hiccup is that the first scoring query will be identical to the last rewritten + * query, so we need to take special care to fix that + * + * @param query The scoring query we wish to profile + * @return A ProfileBreakdown for this query + */ + public ProfileBreakdown getQueryBreakdown(Query query) { + int token = currentToken; + + boolean stackEmpty = stack.isEmpty(); + + // If the stack is empty, we are a new root query + if (stackEmpty) { + + // We couldn't find a rewritten query to attach to, so just add it as a + // top-level root. This is just a precaution: it really shouldn't happen. + // We would only get here if a top-level query that never rewrites for some reason. + roots.add(token); + + // Increment the token since we are adding a new node, but notably, do not + // updateParent() because this was added as a root + currentToken += 1; + stack.add(token); + + return addDependencyNode(query, token); + } + + updateParent(token); + + // Increment the token since we are adding a new node + currentToken += 1; + stack.add(token); + + return addDependencyNode(query, token); + } + + /** + * Begin timing a query for a specific Timing context + */ + public void startRewriteTime() { + assert rewriteScratch == 0; + rewriteScratch = System.nanoTime(); + } + + /** + * Halt the timing process and add the elapsed rewriting time. + * startRewriteTime() must be called for a particular context prior to calling + * stopAndAddRewriteTime(), otherwise the elapsed time will be negative and + * nonsensical + * + * @return The elapsed time + */ + public long stopAndAddRewriteTime() { + long time = Math.max(1, System.nanoTime() - rewriteScratch); + rewriteTime += time; + rewriteScratch = 0; + return time; + } + + /** + * Helper method to add a new node to the dependency tree. + * + * Initializes a new list in the dependency tree, saves the query and + * generates a new {@link ProfileBreakdown} to track the timings + * of this query + * + * @param query The query to profile + * @param token The assigned token for this query + * @return A ProfileBreakdown to profile this query + */ + private ProfileBreakdown addDependencyNode(Query query, int token) { + + // Add a new slot in the dependency tree + tree.add(new ArrayList<>(5)); + + // Save our query for lookup later + queries.add(query); + + ProfileBreakdown queryTimings = new ProfileBreakdown(); + timings.add(token, queryTimings); + return queryTimings; + } + + /** + * Removes the last (e.g. most recent) value on the stack + */ + public void pollLast() { + stack.pollLast(); + } + + /** + * After the query has been run and profiled, we need to merge the flat timing map + * with the dependency graph to build a data structure that mirrors the original + * query tree + * + * @return a hierarchical representation of the profiled query tree + */ + public List getQueryTree() { + ArrayList results = new ArrayList<>(5); + for (Integer root : roots) { + results.add(doGetQueryTree(root)); + } + return results; + } + + /** + * Recursive helper to finalize a node in the dependency tree + * @param token The node we are currently finalizing + * @return A hierarchical representation of the tree inclusive of children at this level + */ + private ProfileResult doGetQueryTree(int token) { + Query query = queries.get(token); + ProfileBreakdown breakdown = timings.get(token); + Map timings = breakdown.toTimingMap(); + List children = tree.get(token); + List childrenProfileResults = Collections.emptyList(); + + if (children != null) { + childrenProfileResults = new ArrayList<>(children.size()); + for (Integer child : children) { + ProfileResult childNode = doGetQueryTree(child); + childrenProfileResults.add(childNode); + } + } + + // TODO this would be better done bottom-up instead of top-down to avoid + // calculating the same times over and over...but worth the effort? + long nodeTime = getNodeTime(timings, childrenProfileResults); + String queryDescription = query.getClass().getSimpleName(); + String luceneName = query.toString(); + return new ProfileResult(queryDescription, luceneName, timings, childrenProfileResults, nodeTime); + } + + public long getRewriteTime() { + return rewriteTime; + } + + /** + * Internal helper to add a child to the current parent node + * + * @param childToken The child to add to the current parent + */ + private void updateParent(int childToken) { + Integer parent = stack.peekLast(); + ArrayList parentNode = tree.get(parent); + parentNode.add(childToken); + tree.set(parent, parentNode); + } + + /** + * Internal helper to calculate the time of a node, inclusive of children + * + * @param timings A map of breakdown timing for the node + * @param children All children profile results at this node + * @return The total time at this node, inclusive of children + */ + private static long getNodeTime(Map timings, List children) { + long nodeTime = 0; + for (long time : timings.values()) { + nodeTime += time; + } + + // Then add up our children + for (ProfileResult child : children) { + nodeTime += getNodeTime(child.getTimeBreakdown(), child.getProfiledChildren()); + } + return nodeTime; + } +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileBreakdown.java b/core/src/main/java/org/elasticsearch/search/profile/ProfileBreakdown.java new file mode 100644 index 00000000000..55ad77b6937 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/ProfileBreakdown.java @@ -0,0 +1,113 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; + +/** + * A record of timings for the various operations that may happen during query execution. + * A node's time may be composed of several internal attributes (rewriting, weighting, + * scoring, etc). + */ +public final class ProfileBreakdown { + + /** Enumeration of all supported timing types. */ + public enum TimingType { + CREATE_WEIGHT, + BUILD_SCORER, + NEXT_DOC, + ADVANCE, + MATCH, + SCORE; + + @Override + public String toString() { + return name().toLowerCase(Locale.ROOT); + } + } + + /** + * The accumulated timings for this query node + */ + private final long[] timings; + + /** Scrach to store the current timing type. */ + private TimingType currentTimingType; + + /** + * The temporary scratch space for holding start-times + */ + private long scratch; + + /** Sole constructor. */ + public ProfileBreakdown() { + timings = new long[TimingType.values().length]; + } + + /** + * Begin timing a query for a specific Timing context + * @param timing The timing context being profiled + */ + public void startTime(TimingType timing) { + assert currentTimingType == null; + assert scratch == 0; + currentTimingType = timing; + scratch = System.nanoTime(); + } + + /** + * Halt the timing process and save the elapsed time. + * startTime() must be called for a particular context prior to calling + * stopAndRecordTime(), otherwise the elapsed time will be negative and + * nonsensical + * + * @return The elapsed time + */ + public long stopAndRecordTime() { + long time = Math.max(1, System.nanoTime() - scratch); + timings[currentTimingType.ordinal()] += time; + currentTimingType = null; + scratch = 0L; + return time; + } + + /** Convert this record to a map from {@link TimingType} to times. */ + public Map toTimingMap() { + Map map = new HashMap<>(); + for (TimingType timingType : TimingType.values()) { + map.put(timingType.toString(), timings[timingType.ordinal()]); + } + return Collections.unmodifiableMap(map); + } + + /** + * Add other's timings into this breakdown + * @param other Another Breakdown to merge with this one + */ + public void merge(ProfileBreakdown other) { + assert(timings.length == other.timings.length); + for (int i = 0; i < timings.length; ++i) { + timings[i] += other.timings[i]; + } + } +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileCollector.java b/core/src/main/java/org/elasticsearch/search/profile/ProfileCollector.java new file mode 100644 index 00000000000..7d7538c9117 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/ProfileCollector.java @@ -0,0 +1,94 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.FilterCollector; +import org.apache.lucene.search.FilterLeafCollector; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Scorer; + +import java.io.IOException; + +/** A collector that profiles how much time is spent calling it. */ +final class ProfileCollector extends FilterCollector { + + private long time; + + /** Sole constructor. */ + public ProfileCollector(Collector in) { + super(in); + } + + /** Return the wrapped collector. */ + public Collector getDelegate() { + return in; + } + + @Override + public boolean needsScores() { + final long start = System.nanoTime(); + try { + return super.needsScores(); + } finally { + time += Math.max(1, System.nanoTime() - start); + } + } + + @Override + public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { + final long start = System.nanoTime(); + final LeafCollector inLeafCollector; + try { + inLeafCollector = super.getLeafCollector(context); + } finally { + time += Math.max(1, System.nanoTime() - start); + } + return new FilterLeafCollector(inLeafCollector) { + + @Override + public void collect(int doc) throws IOException { + final long start = System.nanoTime(); + try { + super.collect(doc); + } finally { + time += Math.max(1, System.nanoTime() - start); + } + } + + @Override + public void setScorer(Scorer scorer) throws IOException { + final long start = System.nanoTime(); + try { + super.setScorer(scorer); + } finally { + time += Math.max(1, System.nanoTime() - start); + } + } + }; + } + + /** Return the total time spent on this collector. */ + public long getTime() { + return time; + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileResult.java b/core/src/main/java/org/elasticsearch/search/profile/ProfileResult.java new file mode 100644 index 00000000000..4c8752fdaf2 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/ProfileResult.java @@ -0,0 +1,165 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +/** + * This class is the internal representation of a profiled Query, corresponding + * to a single node in the query tree. It is built after the query has finished executing + * and is merely a structured representation, rather than the entity that collects the timing + * profile (see InternalProfiler for that) + * + * Each InternalProfileResult has a List of InternalProfileResults, which will contain + * "children" queries if applicable + */ +final class ProfileResult implements Writeable, ToXContent { + + private static final ParseField QUERY_TYPE = new ParseField("query_type"); + private static final ParseField LUCENE_DESCRIPTION = new ParseField("lucene"); + private static final ParseField NODE_TIME = new ParseField("time"); + private static final ParseField CHILDREN = new ParseField("children"); + private static final ParseField BREAKDOWN = new ParseField("breakdown"); + + private final String queryType; + private final String luceneDescription; + private final Map timings; + private final long nodeTime; + private final List children; + + public ProfileResult(String queryType, String luceneDescription, Map timings, List children, long nodeTime) { + this.queryType = queryType; + this.luceneDescription = luceneDescription; + this.timings = timings; + this.children = children; + this.nodeTime = nodeTime; + } + + public ProfileResult(StreamInput in) throws IOException{ + this.queryType = in.readString(); + this.luceneDescription = in.readString(); + this.nodeTime = in.readLong(); + + int timingsSize = in.readVInt(); + this.timings = new HashMap<>(timingsSize); + for (int i = 0; i < timingsSize; ++i) { + timings.put(in.readString(), in.readLong()); + } + + int size = in.readVInt(); + this.children = new ArrayList<>(size); + + for (int i = 0; i < size; i++) { + children.add(new ProfileResult(in)); + } + } + + /** + * Retrieve the lucene description of this query (e.g. the "explain" text) + */ + public String getLuceneDescription() { + return luceneDescription; + } + + /** + * Retrieve the name of the query (e.g. "TermQuery") + */ + public String getQueryName() { + return queryType; + } + + /** + * Returns the timing breakdown for this particular query node + */ + public Map getTimeBreakdown() { + return Collections.unmodifiableMap(timings); + } + + /** + * Returns the total time (inclusive of children) for this query node. + * + * @return elapsed time in nanoseconds + */ + public long getTime() { + return nodeTime; + } + + /** + * Returns a list of all profiled children queries + */ + public List getProfiledChildren() { + return Collections.unmodifiableList(children); + } + + @Override + public ProfileResult readFrom(StreamInput in) throws IOException { + return new ProfileResult(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(queryType); + out.writeString(luceneDescription); + out.writeLong(nodeTime); // not Vlong because can be negative + out.writeVInt(timings.size()); + for (Map.Entry entry : timings.entrySet()) { + out.writeString(entry.getKey()); + out.writeLong(entry.getValue()); + } + out.writeVInt(children.size()); + for (ProfileResult child : children) { + child.writeTo(out); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder = builder.startObject() + .field(QUERY_TYPE.getPreferredName(), queryType) + .field(LUCENE_DESCRIPTION.getPreferredName(), luceneDescription) + .field(NODE_TIME.getPreferredName(), String.format(Locale.US, "%.10gms", (double)(getTime() / 1000000.0))) + .field(BREAKDOWN.getPreferredName(), timings); + + if (!children.isEmpty()) { + builder = builder.startArray(CHILDREN.getPreferredName()); + for (ProfileResult child : children) { + builder = child.toXContent(builder, params); + } + builder = builder.endArray(); + } + + builder = builder.endObject(); + return builder; + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileScorer.java b/core/src/main/java/org/elasticsearch/search/profile/ProfileScorer.java new file mode 100644 index 00000000000..b0dc6f2cd4e --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/ProfileScorer.java @@ -0,0 +1,158 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.TwoPhaseIterator; +import org.apache.lucene.search.Weight; + +import java.io.IOException; +import java.util.Collection; + +/** + * {@link Scorer} wrapper that will compute how much time is spent on moving + * the iterator, confirming matches and computing scores. + */ +final class ProfileScorer extends Scorer { + + private final Scorer scorer; + private ProfileWeight profileWeight; + private final ProfileBreakdown profile; + + ProfileScorer(ProfileWeight w, Scorer scorer, ProfileBreakdown profile) throws IOException { + super(w); + this.scorer = scorer; + this.profileWeight = w; + this.profile = profile; + } + + @Override + public int docID() { + return scorer.docID(); + } + + @Override + public int advance(int target) throws IOException { + profile.startTime(ProfileBreakdown.TimingType.ADVANCE); + try { + return scorer.advance(target); + } finally { + profile.stopAndRecordTime(); + } + } + + @Override + public int nextDoc() throws IOException { + profile.startTime(ProfileBreakdown.TimingType.NEXT_DOC); + try { + return scorer.nextDoc(); + } finally { + profile.stopAndRecordTime(); + } + } + + @Override + public float score() throws IOException { + profile.startTime(ProfileBreakdown.TimingType.SCORE); + try { + return scorer.score(); + } finally { + profile.stopAndRecordTime(); + } + } + + @Override + public int freq() throws IOException { + return scorer.freq(); + } + + @Override + public long cost() { + return scorer.cost(); + } + + @Override + public Weight getWeight() { + return profileWeight; + } + + @Override + public Collection getChildren() { + return scorer.getChildren(); + } + + @Override + public TwoPhaseIterator asTwoPhaseIterator() { + final TwoPhaseIterator in = scorer.asTwoPhaseIterator(); + if (in == null) { + return null; + } + final DocIdSetIterator inApproximation = in.approximation(); + final DocIdSetIterator approximation = new DocIdSetIterator() { + + @Override + public int advance(int target) throws IOException { + profile.startTime(ProfileBreakdown.TimingType.ADVANCE); + try { + return inApproximation.advance(target); + } finally { + profile.stopAndRecordTime(); + } + } + + @Override + public int nextDoc() throws IOException { + profile.startTime(ProfileBreakdown.TimingType.NEXT_DOC); + try { + return inApproximation.nextDoc(); + } finally { + profile.stopAndRecordTime(); + } + } + + @Override + public int docID() { + return inApproximation.docID(); + } + + @Override + public long cost() { + return inApproximation.cost(); + } + }; + return new TwoPhaseIterator(approximation) { + @Override + public boolean matches() throws IOException { + profile.startTime(ProfileBreakdown.TimingType.MATCH); + try { + return in.matches(); + } finally { + profile.stopAndRecordTime(); + } + } + + @Override + public float matchCost() { + return in.matchCost(); + } + }; + } +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileShardResult.java b/core/src/main/java/org/elasticsearch/search/profile/ProfileShardResult.java new file mode 100644 index 00000000000..6e005babb3c --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/ProfileShardResult.java @@ -0,0 +1,103 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.*; + +/** + * A container class to hold the profile results for a single shard in the request. + * Contains a list of query profiles, a collector tree and a total rewrite tree. + */ +public final class ProfileShardResult implements Writeable, ToXContent { + + private final List profileResults; + + private final CollectorResult profileCollector; + + private final long rewriteTime; + + public ProfileShardResult(List profileResults, long rewriteTime, + CollectorResult profileCollector) { + assert(profileCollector != null); + this.profileResults = profileResults; + this.profileCollector = profileCollector; + this.rewriteTime = rewriteTime; + } + + public ProfileShardResult(StreamInput in) throws IOException { + int profileSize = in.readVInt(); + profileResults = new ArrayList<>(profileSize); + for (int j = 0; j < profileSize; j++) { + profileResults.add(new ProfileResult(in)); + } + + profileCollector = new CollectorResult(in); + rewriteTime = in.readLong(); + } + + public List getQueryResults() { + return Collections.unmodifiableList(profileResults); + } + + public long getRewriteTime() { + return rewriteTime; + } + + public CollectorResult getCollectorResult() { + return profileCollector; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startArray("query"); + for (ProfileResult p : profileResults) { + p.toXContent(builder, params); + } + builder.endArray(); + builder.field("rewrite_time", rewriteTime); + builder.startArray("collector"); + profileCollector.toXContent(builder, params); + builder.endArray(); + return builder; + } + + @Override + public ProfileShardResult readFrom(StreamInput in) throws IOException { + return new ProfileShardResult(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(profileResults.size()); + for (ProfileResult p : profileResults) { + p.writeTo(out); + } + profileCollector.writeTo(out); + out.writeLong(rewriteTime); + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/ProfileWeight.java b/core/src/main/java/org/elasticsearch/search/profile/ProfileWeight.java new file mode 100644 index 00000000000..1ce5cd721fe --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/ProfileWeight.java @@ -0,0 +1,97 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BulkScorer; +import org.apache.lucene.search.Explanation; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; + +import java.io.IOException; +import java.util.Set; + +/** + * Weight wrapper that will compute how much time it takes to build the + * {@link Scorer} and then return a {@link Scorer} that is wrapped in + * order to compute timings as well. + */ +public final class ProfileWeight extends Weight { + + private final Weight subQueryWeight; + private final ProfileBreakdown profile; + + public ProfileWeight(Query query, Weight subQueryWeight, ProfileBreakdown profile) throws IOException { + super(query); + this.subQueryWeight = subQueryWeight; + this.profile = profile; + } + + @Override + public Scorer scorer(LeafReaderContext context) throws IOException { + profile.startTime(ProfileBreakdown.TimingType.BUILD_SCORER); + final Scorer subQueryScorer; + try { + subQueryScorer = subQueryWeight.scorer(context); + } finally { + profile.stopAndRecordTime(); + } + if (subQueryScorer == null) { + return null; + } + + return new ProfileScorer(this, subQueryScorer, profile); + } + + @Override + public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { + // We use the default bulk scorer instead of the specialized one. The reason + // is that Lucene's BulkScorers do everything at once: finding matches, + // scoring them and calling the collector, so they make it impossible to + // see where time is spent, which is the purpose of query profiling. + // The default bulk scorer will pull a scorer and iterate over matches, + // this might be a significantly different execution path for some queries + // like disjunctions, but in general this is what is done anyway + return super.bulkScorer(context); + } + + @Override + public Explanation explain(LeafReaderContext context, int doc) throws IOException { + return subQueryWeight.explain(context, doc); + } + + @Override + public float getValueForNormalization() throws IOException { + return subQueryWeight.getValueForNormalization(); + } + + @Override + public void normalize(float norm, float topLevelBoost) { + subQueryWeight.normalize(norm, topLevelBoost); + } + + @Override + public void extractTerms(Set set) { + subQueryWeight.extractTerms(set); + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/Profiler.java b/core/src/main/java/org/elasticsearch/search/profile/Profiler.java new file mode 100644 index 00000000000..bf0c9ec01b6 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/Profiler.java @@ -0,0 +1,130 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.apache.lucene.search.Query; + +import java.util.*; + +/** + * This class acts as a thread-local storage for profiling a query. It also + * builds a representation of the query tree which is built constructed + * "online" as the weights are wrapped by ContextIndexSearcher. This allows us + * to know the relationship between nodes in tree without explicitly + * walking the tree or pre-wrapping everything + * + * A Profiler is associated with every Search, not per Search-Request. E.g. a + * request may execute two searches (query + global agg). A Profiler just + * represents one of those + */ +public final class Profiler { + + private final InternalProfileTree queryTree = new InternalProfileTree(); + + /** + * The root Collector used in the search + */ + private InternalProfileCollector collector; + + public Profiler() {} + + /** Set the collector that is associated with this profiler. */ + public void setCollector(InternalProfileCollector collector) { + if (this.collector != null) { + throw new IllegalStateException("The collector can only be set once."); + } + this.collector = Objects.requireNonNull(collector); + } + + /** + * Get the {@link ProfileBreakdown} for the given query, potentially creating it if it did not exist. + * This should only be used for queries that will be undergoing scoring. Do not use it to profile the + * rewriting phase + */ + public ProfileBreakdown getQueryBreakdown(Query query) { + return queryTree.getQueryBreakdown(query); + } + + /** + * Begin timing the rewrite phase of a request. All rewrites are accumulated together into a + * single metric + */ + public void startRewriteTime() { + queryTree.startRewriteTime(); + } + + /** + * Stop recording the current rewrite and add it's time to the total tally, returning the + * cumulative time so far. + * + * @return cumulative rewrite time + */ + public long stopAndAddRewriteTime() { + return queryTree.stopAndAddRewriteTime(); + } + + /** + * Removes the last (e.g. most recent) query on the stack. This should only be called for scoring + * queries, not rewritten queries + */ + public void pollLastQuery() { + queryTree.pollLast(); + } + + /** + * @return a hierarchical representation of the profiled query tree + */ + public List getQueryTree() { + return queryTree.getQueryTree(); + } + + /** + * @return total time taken to rewrite all queries in this profile + */ + public long getRewriteTime() { + return queryTree.getRewriteTime(); + } + + /** + * Return the current root Collector for this search + */ + public CollectorResult getCollector() { + return collector.getCollectorTree(); + } + + /** + * Helper method to convert Profiler into InternalProfileShardResults, which can be + * serialized to other nodes, emitted as JSON, etc. + * + * @param profilers A list of Profilers to convert into InternalProfileShardResults + * @return A list of corresponding InternalProfileShardResults + */ + public static List buildShardResults(List profilers) { + List results = new ArrayList<>(profilers.size()); + for (Profiler profiler : profilers) { + ProfileShardResult result = new ProfileShardResult( + profiler.getQueryTree(), profiler.getRewriteTime(), profiler.getCollector()); + results.add(result); + } + return results; + } + + +} diff --git a/core/src/main/java/org/elasticsearch/search/profile/Profilers.java b/core/src/main/java/org/elasticsearch/search/profile/Profilers.java new file mode 100644 index 00000000000..0fb7d9ac1c9 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/profile/Profilers.java @@ -0,0 +1,59 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.elasticsearch.search.internal.ContextIndexSearcher; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** Wrapper around several {@link Profiler}s that makes management easier. */ +public final class Profilers { + + private final ContextIndexSearcher searcher; + private final List profilers; + + /** Sole constructor. This {@link Profilers} instance will initiall wrap one {@link Profiler}. */ + public Profilers(ContextIndexSearcher searcher) { + this.searcher = searcher; + this.profilers = new ArrayList<>(); + addProfiler(); + } + + /** Switch to a new profile. */ + public Profiler addProfiler() { + Profiler profiler = new Profiler(); + searcher.setProfiler(profiler); + profilers.add(profiler); + return profiler; + } + + /** Get the current profiler. */ + public Profiler getCurrent() { + return profilers.get(profilers.size() - 1); + } + + /** Return the list of all created {@link Profiler}s so far. */ + public List getProfilers() { + return Collections.unmodifiableList(profilers); + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java b/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java index ce8836cd336..08ff849871f 100644 --- a/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java +++ b/core/src/main/java/org/elasticsearch/search/query/QueryPhase.java @@ -52,13 +52,16 @@ import org.elasticsearch.search.SearchService; import org.elasticsearch.search.aggregations.AggregationPhase; import org.elasticsearch.search.internal.ScrollContext; import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.profile.*; import org.elasticsearch.search.rescore.RescorePhase; import org.elasticsearch.search.rescore.RescoreSearchContext; import org.elasticsearch.search.sort.SortParseElement; import org.elasticsearch.search.sort.TrackScoresParseElement; import org.elasticsearch.search.suggest.SuggestPhase; +import java.util.AbstractList; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -124,6 +127,11 @@ public class QueryPhase implements SearchPhase { } suggestPhase.execute(searchContext); aggregationPhase.execute(searchContext); + + if (searchContext.getProfilers() != null) { + List shardResults = Profiler.buildShardResults(searchContext.getProfilers().getProfilers()); + searchContext.queryResult().profileResults(shardResults); + } } private static boolean returnsDocsInOrder(Query query, Sort sort) { @@ -147,6 +155,7 @@ public class QueryPhase implements SearchPhase { QuerySearchResult queryResult = searchContext.queryResult(); queryResult.searchTimedOut(false); + final boolean doProfile = searchContext.getProfilers() != null; final SearchType searchType = searchContext.searchType(); boolean rescore = false; try { @@ -162,9 +171,13 @@ public class QueryPhase implements SearchPhase { Callable topDocsCallable; assert query == searcher.rewrite(query); // already rewritten + if (searchContext.size() == 0) { // no matter what the value of from is final TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector(); collector = totalHitCountCollector; + if (searchContext.getProfilers() != null) { + collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_COUNT, Collections.emptyList()); + } topDocsCallable = new Callable() { @Override public TopDocs call() throws Exception { @@ -219,6 +232,9 @@ public class QueryPhase implements SearchPhase { topDocsCollector = TopScoreDocCollector.create(numDocs, lastEmittedDoc); } collector = topDocsCollector; + if (doProfile) { + collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_TOP_HITS, Collections.emptyList()); + } topDocsCallable = new Callable() { @Override public TopDocs call() throws Exception { @@ -254,27 +270,57 @@ public class QueryPhase implements SearchPhase { final boolean terminateAfterSet = searchContext.terminateAfter() != SearchContext.DEFAULT_TERMINATE_AFTER; if (terminateAfterSet) { + final Collector child = collector; // throws Lucene.EarlyTerminationException when given count is reached collector = Lucene.wrapCountBasedEarlyTerminatingCollector(collector, searchContext.terminateAfter()); + if (doProfile) { + collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_TERMINATE_AFTER_COUNT, + Collections.singletonList((InternalProfileCollector) child)); + } } if (searchContext.parsedPostFilter() != null) { + final Collector child = collector; // this will only get applied to the actual search collector and not // to any scoped collectors, also, it will only be applied to the main collector // since that is where the filter should only work final Weight filterWeight = searcher.createNormalizedWeight(searchContext.parsedPostFilter().query(), false); collector = new FilteredCollector(collector, filterWeight); + if (doProfile) { + collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_POST_FILTER, + Collections.singletonList((InternalProfileCollector) child)); + } } // plug in additional collectors, like aggregations - List allCollectors = new ArrayList<>(); - allCollectors.add(collector); - allCollectors.addAll(searchContext.queryCollectors().values()); - collector = MultiCollector.wrap(allCollectors); + final List subCollectors = new ArrayList<>(); + subCollectors.add(collector); + subCollectors.addAll(searchContext.queryCollectors().values()); + collector = MultiCollector.wrap(subCollectors); + if (doProfile && collector instanceof InternalProfileCollector == false) { + // When there is a single collector to wrap, MultiCollector returns it + // directly, so only wrap in the case that there are several sub collectors + final List children = new AbstractList() { + @Override + public InternalProfileCollector get(int index) { + return (InternalProfileCollector) subCollectors.get(index); + } + @Override + public int size() { + return subCollectors.size(); + } + }; + collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_MULTI, children); + } // apply the minimum score after multi collector so we filter aggs as well if (searchContext.minimumScore() != null) { + final Collector child = collector; collector = new MinimumScoreCollector(collector, searchContext.minimumScore()); + if (doProfile) { + collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_MIN_SCORE, + Collections.singletonList((InternalProfileCollector) child)); + } } if (collector.getClass() == TotalHitCountCollector.class) { @@ -319,13 +365,21 @@ public class QueryPhase implements SearchPhase { final boolean timeoutSet = searchContext.timeoutInMillis() != SearchService.NO_TIMEOUT.millis(); if (timeoutSet && collector != null) { // collector might be null if no collection is actually needed + final Collector child = collector; // TODO: change to use our own counter that uses the scheduler in ThreadPool // throws TimeLimitingCollector.TimeExceededException when timeout has reached collector = Lucene.wrapTimeLimitingCollector(collector, searchContext.timeEstimateCounter(), searchContext.timeoutInMillis()); + if (doProfile) { + collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_TIMEOUT, + Collections.singletonList((InternalProfileCollector) child)); + } } try { if (collector != null) { + if (doProfile) { + searchContext.getProfilers().getCurrent().setCollector((InternalProfileCollector) collector); + } searcher.search(query, collector); } } catch (TimeLimitingCollector.TimeExceededException e) { @@ -343,7 +397,13 @@ public class QueryPhase implements SearchPhase { queryResult.topDocs(topDocsCallable.call()); + if (searchContext.getProfilers() != null) { + List shardResults = Profiler.buildShardResults(searchContext.getProfilers().getProfilers()); + searchContext.queryResult().profileResults(shardResults); + } + return rescore; + } catch (Throwable e) { throw new QueryPhaseExecutionException(searchContext, "Failed to execute main query", e); } diff --git a/core/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java b/core/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java index 7f8d12a9c90..9223eb5a82d 100644 --- a/core/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/core/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -20,6 +20,8 @@ package org.elasticsearch.search.query; import org.apache.lucene.search.TopDocs; +import org.elasticsearch.Version; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -29,6 +31,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams; import org.elasticsearch.search.aggregations.pipeline.SiblingPipelineAggregator; +import org.elasticsearch.search.profile.ProfileShardResult; import org.elasticsearch.search.suggest.Suggest; import java.io.IOException; @@ -53,6 +56,7 @@ public class QuerySearchResult extends QuerySearchResultProvider { private Suggest suggest; private boolean searchTimedOut; private Boolean terminatedEarly = null; + private List profileShardResults; public QuerySearchResult() { @@ -120,6 +124,22 @@ public class QuerySearchResult extends QuerySearchResultProvider { this.aggregations = aggregations; } + /** + * Returns the profiled results for this search, or potentially null if result was empty + * @return The profiled results, or null + */ + public @Nullable List profileResults() { + return profileShardResults; + } + + /** + * Sets the finalized profiling results for this query + * @param shardResults The finalized profile + */ + public void profileResults(List shardResults) { + this.profileShardResults = shardResults; + } + public List pipelineAggregators() { return pipelineAggregators; } @@ -191,6 +211,15 @@ public class QuerySearchResult extends QuerySearchResultProvider { } searchTimedOut = in.readBoolean(); terminatedEarly = in.readOptionalBoolean(); + + if (in.getVersion().onOrAfter(Version.V_2_2_0) && in.readBoolean()) { + int profileSize = in.readVInt(); + profileShardResults = new ArrayList<>(profileSize); + for (int i = 0; i < profileSize; i++) { + ProfileShardResult result = new ProfileShardResult(in); + profileShardResults.add(result); + } + } } @Override @@ -229,5 +258,17 @@ public class QuerySearchResult extends QuerySearchResultProvider { } out.writeBoolean(searchTimedOut); out.writeOptionalBoolean(terminatedEarly); + + if (out.getVersion().onOrAfter(Version.V_2_2_0)) { + if (profileShardResults == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeVInt(profileShardResults.size()); + for (ProfileShardResult shardResult : profileShardResults) { + shardResult.writeTo(out); + } + } + } } } diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java index 90d4437fcea..207b626409b 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java @@ -57,6 +57,7 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.min; import static org.elasticsearch.search.aggregations.AggregationBuilders.range; import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilders.derivative; import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilders.movingAvg; +import static org.elasticsearch.test.hamcrest.DoubleMatcher.nearlyEqual; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; diff --git a/core/src/test/java/org/elasticsearch/search/profile/ProfileTests.java b/core/src/test/java/org/elasticsearch/search/profile/ProfileTests.java new file mode 100644 index 00000000000..83f6efaa150 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/profile/ProfileTests.java @@ -0,0 +1,173 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field.Store; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.RandomApproximationQuery; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.TotalHitCountCollector; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.TestUtil; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.search.internal.ContextIndexSearcher; +import org.elasticsearch.test.ESTestCase; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; + +public class ProfileTests extends ESTestCase { + + static Directory dir; + static IndexReader reader; + static ContextIndexSearcher searcher; + + @BeforeClass + public static void before() throws IOException { + dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + final int numDocs = TestUtil.nextInt(random(), 1, 20); + for (int i = 0; i < numDocs; ++i) { + final int numHoles = random().nextInt(5); + for (int j = 0; j < numHoles; ++j) { + w.addDocument(new Document()); + } + Document doc = new Document(); + doc.add(new StringField("foo", "bar", Store.NO)); + w.addDocument(doc); + } + reader = w.getReader(); + w.close(); + Engine.Searcher engineSearcher = new Engine.Searcher("test", new IndexSearcher(reader)); + searcher = new ContextIndexSearcher(engineSearcher, IndexSearcher.getDefaultQueryCache(), MAYBE_CACHE_POLICY); + } + + @AfterClass + public static void after() throws IOException { + IOUtils.close(reader, dir); + dir = null; + reader = null; + searcher = null; + } + + public void testBasic() throws IOException { + Profiler profiler = new Profiler(); + searcher.setProfiler(profiler); + Query query = new TermQuery(new Term("foo", "bar")); + searcher.search(query, 1); + List results = profiler.getQueryTree(); + assertEquals(1, results.size()); + Map breakdown = results.get(0).getTimeBreakdown(); + assertThat(breakdown.get(ProfileBreakdown.TimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.BUILD_SCORER.toString()).longValue(), greaterThan(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.NEXT_DOC.toString()).longValue(), greaterThan(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.ADVANCE.toString()).longValue(), equalTo(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.SCORE.toString()).longValue(), greaterThan(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.MATCH.toString()).longValue(), equalTo(0L)); + + long rewriteTime = profiler.getRewriteTime(); + assertThat(rewriteTime, greaterThan(0L)); + } + + public void testNoScoring() throws IOException { + Profiler profiler = new Profiler(); + searcher.setProfiler(profiler); + Query query = new TermQuery(new Term("foo", "bar")); + searcher.search(query, 1, Sort.INDEXORDER); // scores are not needed + List results = profiler.getQueryTree(); + assertEquals(1, results.size()); + Map breakdown = results.get(0).getTimeBreakdown(); + assertThat(breakdown.get(ProfileBreakdown.TimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.BUILD_SCORER.toString()).longValue(), greaterThan(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.NEXT_DOC.toString()).longValue(), greaterThan(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.ADVANCE.toString()).longValue(), equalTo(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.SCORE.toString()).longValue(), equalTo(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.MATCH.toString()).longValue(), equalTo(0L)); + + long rewriteTime = profiler.getRewriteTime(); + assertThat(rewriteTime, greaterThan(0L)); + } + + public void testUseIndexStats() throws IOException { + Profiler profiler = new Profiler(); + searcher.setProfiler(profiler); + Query query = new TermQuery(new Term("foo", "bar")); + searcher.count(query); // will use index stats + List results = profiler.getQueryTree(); + assertEquals(0, results.size()); + + long rewriteTime = profiler.getRewriteTime(); + assertThat(rewriteTime, greaterThan(0L)); + } + + public void testApproximations() throws IOException { + Profiler profiler = new Profiler(); + Engine.Searcher engineSearcher = new Engine.Searcher("test", new IndexSearcher(reader)); + // disable query caching since we want to test approximations, which won't + // be exposed on a cached entry + ContextIndexSearcher searcher = new ContextIndexSearcher(engineSearcher, null, MAYBE_CACHE_POLICY); + searcher.setProfiler(profiler); + Query query = new RandomApproximationQuery(new TermQuery(new Term("foo", "bar")), random()); + searcher.count(query); + List results = profiler.getQueryTree(); + assertEquals(1, results.size()); + Map breakdown = results.get(0).getTimeBreakdown(); + assertThat(breakdown.get(ProfileBreakdown.TimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.BUILD_SCORER.toString()).longValue(), greaterThan(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.NEXT_DOC.toString()).longValue(), greaterThan(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.ADVANCE.toString()).longValue(), equalTo(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.SCORE.toString()).longValue(), equalTo(0L)); + assertThat(breakdown.get(ProfileBreakdown.TimingType.MATCH.toString()).longValue(), greaterThan(0L)); + + long rewriteTime = profiler.getRewriteTime(); + assertThat(rewriteTime, greaterThan(0L)); + + } + + public void testCollector() throws IOException { + TotalHitCountCollector collector = new TotalHitCountCollector(); + ProfileCollector profileCollector = new ProfileCollector(collector); + assertEquals(0, profileCollector.getTime()); + final LeafCollector leafCollector = profileCollector.getLeafCollector(reader.leaves().get(0)); + assertThat(profileCollector.getTime(), greaterThan(0L)); + long time = profileCollector.getTime(); + leafCollector.setScorer(Lucene.illegalScorer("dummy scorer")); + assertThat(profileCollector.getTime(), greaterThan(time)); + time = profileCollector.getTime(); + leafCollector.collect(0); + assertThat(profileCollector.getTime(), greaterThan(time)); + } +} diff --git a/core/src/test/java/org/elasticsearch/search/profile/QueryProfilerIT.java b/core/src/test/java/org/elasticsearch/search/profile/QueryProfilerIT.java new file mode 100644 index 00000000000..bb33364a751 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/profile/QueryProfilerIT.java @@ -0,0 +1,596 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.elasticsearch.action.search.*; +import org.elasticsearch.search.SearchHit; +import org.apache.lucene.util.English; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.index.query.*; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.test.ESIntegTestCase; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.search.profile.RandomQueryGenerator.randomQueryBuilder; +import static org.elasticsearch.test.hamcrest.DoubleMatcher.nearlyEqual; +import static org.hamcrest.Matchers.*; + + +public class QueryProfilerIT extends ESIntegTestCase { + + /** + * This test simply checks to make sure nothing crashes. Test indexes 100-150 documents, + * constructs 20-100 random queries and tries to profile them + */ + public void testProfileQuery() throws Exception { + createIndex("test"); + ensureGreen(); + + int numDocs = randomIntBetween(100, 150); + IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + docs[i] = client().prepareIndex("test", "type1", String.valueOf(i)).setSource( + "field1", English.intToEnglish(i), + "field2", i + ); + } + + List stringFields = Arrays.asList("field1"); + List numericFields = Arrays.asList("field2"); + + indexRandom(true, docs); + + refresh(); + int iters = between(20, 100); + for (int i = 0; i < iters; i++) { + QueryBuilder q = randomQueryBuilder(stringFields, numericFields, numDocs, 3); + logger.info(q.toString()); + + SearchResponse resp = client().prepareSearch() + .setQuery(q) + .setProfile(true) + .setSearchType(SearchType.QUERY_THEN_FETCH) + .execute().actionGet(); + + assertNotNull("Profile response element should not be null", resp.getProfileResults()); + for (Map.Entry> shard : resp.getProfileResults().entrySet()) { + for (ProfileShardResult searchProfiles : shard.getValue()) { + for (ProfileResult result : searchProfiles.getQueryResults()) { + assertNotNull(result.getQueryName()); + assertNotNull(result.getLuceneDescription()); + assertThat(result.getTime(), greaterThan(0L)); + } + + CollectorResult result = searchProfiles.getCollectorResult(); + assertThat(result.getName(), not(isEmptyOrNullString())); + assertThat(result.getTime(), greaterThan(0L)); + } + } + + } + } + + /** + * This test generates 1-10 random queries and executes a profiled and non-profiled + * search for each query. It then does some basic sanity checking of score and hits + * to make sure the profiling doesn't interfere with the hits being returned + */ + public void testProfileMatchesRegular() throws Exception { + createIndex("test"); + ensureGreen(); + + int numDocs = randomIntBetween(100, 150); + IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + docs[i] = client().prepareIndex("test", "type1", String.valueOf(i)).setSource( + "field1", English.intToEnglish(i), + "field2", i + ); + } + + List stringFields = Arrays.asList("field1"); + List numericFields = Arrays.asList("field2"); + + indexRandom(true, docs); + + refresh(); + int iters = between(1, 10); + for (int i = 0; i < iters; i++) { + QueryBuilder q = randomQueryBuilder(stringFields, numericFields, numDocs, 3); + logger.info(q.toString()); + + + SearchRequestBuilder vanilla = client().prepareSearch("test") + .setQuery(q) + .setProfile(false) + .addSort("_score", SortOrder.DESC) + .addSort("_uid", SortOrder.ASC) + .setPreference("_primary") + .setSearchType(SearchType.QUERY_THEN_FETCH); + + SearchRequestBuilder profile = client().prepareSearch("test") + .setQuery(q) + .setProfile(true) + .addSort("_score", SortOrder.DESC) + .addSort("_uid", SortOrder.ASC) + .setPreference("_primary") + .setSearchType(SearchType.QUERY_THEN_FETCH); + + MultiSearchResponse.Item[] responses = client().prepareMultiSearch() + .add(vanilla) + .add(profile) + .execute().actionGet().getResponses(); + + SearchResponse vanillaResponse = responses[0].getResponse(); + SearchResponse profileResponse = responses[1].getResponse(); + + float vanillaMaxScore = vanillaResponse.getHits().getMaxScore(); + float profileMaxScore = profileResponse.getHits().getMaxScore(); + if (Float.isNaN(vanillaMaxScore)) { + assertTrue("Vanilla maxScore is NaN but Profile is not [" + profileMaxScore + "]", + Float.isNaN(profileMaxScore)); + } else { + assertTrue("Profile maxScore of [" + profileMaxScore + "] is not close to Vanilla maxScore [" + vanillaMaxScore + "]", + nearlyEqual(vanillaMaxScore, profileMaxScore, 0.001)); + } + + assertThat("Profile totalHits of [" + profileResponse.getHits().totalHits() + "] is not close to Vanilla totalHits [" + vanillaResponse.getHits().totalHits() + "]", + vanillaResponse.getHits().getTotalHits(), equalTo(profileResponse.getHits().getTotalHits())); + + SearchHit[] vanillaHits = vanillaResponse.getHits().getHits(); + SearchHit[] profileHits = profileResponse.getHits().getHits(); + + for (int j = 0; j < vanillaHits.length; j++) { + assertThat("Profile hit #" + j + " has a different ID from Vanilla", + vanillaHits[j].getId(), equalTo(profileHits[j].getId())); + } + + } + } + + /** + * This test verifies that the output is reasonable for a simple, non-nested query + */ + public void testSimpleMatch() throws Exception { + createIndex("test"); + int numDocs = randomIntBetween(100, 150); + IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + docs[i] = client().prepareIndex("test", "type1", String.valueOf(i)).setSource( + "field1", English.intToEnglish(i), + "field2", i + ); + } + + indexRandom(true, docs); + ensureGreen(); + + QueryBuilder q = QueryBuilders.matchQuery("field1", "one"); + + SearchResponse resp = client().prepareSearch() + .setQuery(q) + .setProfile(true) + .setSearchType(SearchType.QUERY_THEN_FETCH) + .execute().actionGet(); + + Map> p = resp.getProfileResults(); + assertNotNull(p); + + for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { + for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (ProfileResult result : searchProfiles.getQueryResults()) { + assertEquals(result.getQueryName(), "TermQuery"); + assertEquals(result.getLuceneDescription(), "field1:one"); + assertThat(result.getTime(), greaterThan(0L)); + assertNotNull(result.getTimeBreakdown()); + } + + CollectorResult result = searchProfiles.getCollectorResult(); + assertThat(result.getName(), not(isEmptyOrNullString())); + assertThat(result.getTime(), greaterThan(0L)); + } + } + } + + /** + * This test verifies that the output is reasonable for a nested query + */ + public void testBool() throws Exception { + createIndex("test"); + ensureGreen(); + + int numDocs = randomIntBetween(100, 150); + IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + docs[i] = client().prepareIndex("test", "type1", String.valueOf(i)).setSource( + "field1", English.intToEnglish(i), + "field2", i + ); + } + + indexRandom(true, docs); + + QueryBuilder q = QueryBuilders.boolQuery().must(QueryBuilders.matchQuery("field1", "one")).must(QueryBuilders.matchQuery("field1", "two")); + + SearchResponse resp = client().prepareSearch() + .setQuery(q) + .setProfile(true) + .setSearchType(SearchType.QUERY_THEN_FETCH) + .execute().actionGet(); + + Map> p = resp.getProfileResults(); + assertNotNull(p); + + for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { + for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (ProfileResult result : searchProfiles.getQueryResults()) { + assertEquals(result.getQueryName(), "BooleanQuery"); + assertEquals(result.getLuceneDescription(), "+field1:one +field1:two"); + assertThat(result.getTime(), greaterThan(0L)); + assertNotNull(result.getTimeBreakdown()); + assertEquals(result.getProfiledChildren().size(), 2); + + // Check the children + List children = result.getProfiledChildren(); + assertEquals(children.size(), 2); + + ProfileResult childProfile = children.get(0); + assertEquals(childProfile.getQueryName(), "TermQuery"); + assertEquals(childProfile.getLuceneDescription(), "field1:one"); + assertThat(childProfile.getTime(), greaterThan(0L)); + assertNotNull(childProfile.getTimeBreakdown()); + assertEquals(childProfile.getProfiledChildren().size(), 0); + + childProfile = children.get(1); + assertEquals(childProfile.getQueryName(), "TermQuery"); + assertEquals(childProfile.getLuceneDescription(), "field1:two"); + assertThat(childProfile.getTime(), greaterThan(0L)); + assertNotNull(childProfile.getTimeBreakdown()); + } + + CollectorResult result = searchProfiles.getCollectorResult(); + assertThat(result.getName(), not(isEmptyOrNullString())); + assertThat(result.getTime(), greaterThan(0L)); + } + } + + + } + + /** + * Tests a boolean query with no children clauses + */ + public void testEmptyBool() throws Exception { + createIndex("test"); + ensureGreen(); + + int numDocs = randomIntBetween(100, 150); + IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + docs[i] = client().prepareIndex("test", "type1", String.valueOf(i)).setSource( + "field1", English.intToEnglish(i), + "field2", i + ); + } + + indexRandom(true, docs); + + refresh(); + + QueryBuilder q = QueryBuilders.boolQuery(); + logger.info(q.toString()); + + SearchResponse resp = client().prepareSearch() + .setQuery(q) + .setProfile(true) + .setSearchType(SearchType.QUERY_THEN_FETCH) + .execute().actionGet(); + + assertNotNull("Profile response element should not be null", resp.getProfileResults()); + + for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { + for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (ProfileResult result : searchProfiles.getQueryResults()) { + assertNotNull(result.getQueryName()); + assertNotNull(result.getLuceneDescription()); + assertThat(result.getTime(), greaterThan(0L)); + assertNotNull(result.getTimeBreakdown()); + } + + CollectorResult result = searchProfiles.getCollectorResult(); + assertThat(result.getName(), not(isEmptyOrNullString())); + assertThat(result.getTime(), greaterThan(0L)); + } + } + } + + /** + * Tests a series of three nested boolean queries with a single "leaf" match query. + * The rewrite process will "collapse" this down to a single bool, so this tests to make sure + * nothing catastrophic happens during that fairly substantial rewrite + */ + public void testCollapsingBool() throws Exception { + createIndex("test"); + ensureGreen(); + + int numDocs = randomIntBetween(100, 150); + IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + docs[i] = client().prepareIndex("test", "type1", String.valueOf(i)).setSource( + "field1", English.intToEnglish(i), + "field2", i + ); + } + + indexRandom(true, docs); + + refresh(); + + QueryBuilder q = QueryBuilders.boolQuery().must(QueryBuilders.boolQuery().must(QueryBuilders.boolQuery().must(QueryBuilders.matchQuery("field1", "one")))); + + + logger.info(q.toString()); + + SearchResponse resp = client().prepareSearch() + .setQuery(q) + .setProfile(true) + .setSearchType(SearchType.QUERY_THEN_FETCH) + .execute().actionGet(); + + assertNotNull("Profile response element should not be null", resp.getProfileResults()); + + for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { + for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (ProfileResult result : searchProfiles.getQueryResults()) { + assertNotNull(result.getQueryName()); + assertNotNull(result.getLuceneDescription()); + assertThat(result.getTime(), greaterThan(0L)); + assertNotNull(result.getTimeBreakdown()); + } + + CollectorResult result = searchProfiles.getCollectorResult(); + assertThat(result.getName(), not(isEmptyOrNullString())); + assertThat(result.getTime(), greaterThan(0L)); + } + } + } + + public void testBoosting() throws Exception { + createIndex("test"); + ensureGreen(); + + int numDocs = randomIntBetween(100, 150); + IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + docs[i] = client().prepareIndex("test", "type1", String.valueOf(i)).setSource( + "field1", English.intToEnglish(i), + "field2", i + ); + } + + indexRandom(true, docs); + + refresh(); + + QueryBuilder q = QueryBuilders.boostingQuery(QueryBuilders.matchQuery("field1", "one"), QueryBuilders.matchQuery("field1", "two")) + .boost(randomFloat()) + .negativeBoost(randomFloat()); + logger.info(q.toString()); + + SearchResponse resp = client().prepareSearch() + .setQuery(q) + .setProfile(true) + .setSearchType(SearchType.QUERY_THEN_FETCH) + .execute().actionGet(); + + assertNotNull("Profile response element should not be null", resp.getProfileResults()); + + for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { + for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (ProfileResult result : searchProfiles.getQueryResults()) { + assertNotNull(result.getQueryName()); + assertNotNull(result.getLuceneDescription()); + assertThat(result.getTime(), greaterThan(0L)); + assertNotNull(result.getTimeBreakdown()); + } + + CollectorResult result = searchProfiles.getCollectorResult(); + assertThat(result.getName(), not(isEmptyOrNullString())); + assertThat(result.getTime(), greaterThan(0L)); + } + } + } + + public void testDisMaxRange() throws Exception { + createIndex("test"); + ensureGreen(); + + int numDocs = randomIntBetween(100, 150); + IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + docs[i] = client().prepareIndex("test", "type1", String.valueOf(i)).setSource( + "field1", English.intToEnglish(i), + "field2", i + ); + } + + indexRandom(true, docs); + + refresh(); + + QueryBuilder q = QueryBuilders.disMaxQuery() + .boost(0.33703882f) + .add(QueryBuilders.rangeQuery("field2").from(null).to(73).includeLower(true).includeUpper(true)); + logger.info(q.toString()); + + SearchResponse resp = client().prepareSearch() + .setQuery(q) + .setProfile(true) + .setSearchType(SearchType.QUERY_THEN_FETCH) + .execute().actionGet(); + + assertNotNull("Profile response element should not be null", resp.getProfileResults()); + + for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { + for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (ProfileResult result : searchProfiles.getQueryResults()) { + assertNotNull(result.getQueryName()); + assertNotNull(result.getLuceneDescription()); + assertThat(result.getTime(), greaterThan(0L)); + assertNotNull(result.getTimeBreakdown()); + } + + CollectorResult result = searchProfiles.getCollectorResult(); + assertThat(result.getName(), not(isEmptyOrNullString())); + assertThat(result.getTime(), greaterThan(0L)); + } + } + } + + public void testRange() throws Exception { + createIndex("test"); + ensureGreen(); + + int numDocs = randomIntBetween(100, 150); + IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + docs[i] = client().prepareIndex("test", "type1", String.valueOf(i)).setSource( + "field1", English.intToEnglish(i), + "field2", i + ); + } + + indexRandom(true, docs); + + refresh(); + + QueryBuilder q = QueryBuilders.rangeQuery("field2").from(0).to(5); + + logger.info(q.toString()); + + SearchResponse resp = client().prepareSearch() + .setQuery(q) + .setProfile(true) + .setSearchType(SearchType.QUERY_THEN_FETCH) + .execute().actionGet(); + + assertNotNull("Profile response element should not be null", resp.getProfileResults()); + + for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { + for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (ProfileResult result : searchProfiles.getQueryResults()) { + assertNotNull(result.getQueryName()); + assertNotNull(result.getLuceneDescription()); + assertThat(result.getTime(), greaterThan(0L)); + assertNotNull(result.getTimeBreakdown()); + } + + CollectorResult result = searchProfiles.getCollectorResult(); + assertThat(result.getName(), not(isEmptyOrNullString())); + assertThat(result.getTime(), greaterThan(0L)); + } + } + } + + public void testPhrase() throws Exception { + createIndex("test"); + ensureGreen(); + + int numDocs = randomIntBetween(100, 150); + IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + docs[i] = client().prepareIndex("test", "type1", String.valueOf(i)).setSource( + "field1", English.intToEnglish(i) + " " + English.intToEnglish(i+1), + "field2", i + ); + } + + indexRandom(true, docs); + + refresh(); + + QueryBuilder q = QueryBuilders.matchPhraseQuery("field1", "one two"); + + logger.info(q.toString()); + + SearchResponse resp = client().prepareSearch() + .setQuery(q) + .setIndices("test") + .setTypes("type1") + .setProfile(true) + .setSearchType(SearchType.QUERY_THEN_FETCH) + .execute().actionGet(); + + if (resp.getShardFailures().length > 0) { + for (ShardSearchFailure f : resp.getShardFailures()) { + logger.error(f.toString()); + } + fail(); + } + + assertNotNull("Profile response element should not be null", resp.getProfileResults()); + + for (Map.Entry> shardResult : resp.getProfileResults().entrySet()) { + for (ProfileShardResult searchProfiles : shardResult.getValue()) { + for (ProfileResult result : searchProfiles.getQueryResults()) { + assertNotNull(result.getQueryName()); + assertNotNull(result.getLuceneDescription()); + assertThat(result.getTime(), greaterThan(0L)); + assertNotNull(result.getTimeBreakdown()); + } + + CollectorResult result = searchProfiles.getCollectorResult(); + assertThat(result.getName(), not(isEmptyOrNullString())); + assertThat(result.getTime(), greaterThan(0L)); + } + } + } + + /** + * This test makes sure no profile results are returned when profiling is disabled + */ + public void testNoProfile() throws Exception { + createIndex("test"); + ensureGreen(); + + int numDocs = randomIntBetween(100, 150); + IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + docs[i] = client().prepareIndex("test", "type1", String.valueOf(i)).setSource( + "field1", English.intToEnglish(i), + "field2", i + ); + } + + indexRandom(true, docs); + refresh(); + QueryBuilder q = QueryBuilders.rangeQuery("field2").from(0).to(5); + + logger.info(q.toString()); + + SearchResponse resp = client().prepareSearch().setQuery(q).setProfile(false).execute().actionGet(); + assertThat("Profile response element should be an empty map", resp.getProfileResults().size(), equalTo(0)); + } + +} + diff --git a/core/src/test/java/org/elasticsearch/search/profile/RandomQueryGenerator.java b/core/src/test/java/org/elasticsearch/search/profile/RandomQueryGenerator.java new file mode 100644 index 00000000000..fb8cd40ce52 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/profile/RandomQueryGenerator.java @@ -0,0 +1,266 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.profile; + +import org.apache.lucene.util.English; +import org.elasticsearch.common.unit.Fuzziness; +import org.elasticsearch.index.query.*; + +import java.util.ArrayList; +import java.util.List; + +import static com.carrotsearch.randomizedtesting.RandomizedTest.*; +import static org.junit.Assert.assertTrue; + + +public class RandomQueryGenerator { + public static QueryBuilder randomQueryBuilder(List stringFields, List numericFields, int numDocs, int depth) { + assertTrue("Must supply at least one string field", stringFields.size() > 0); + assertTrue("Must supply at least one numeric field", numericFields.size() > 0); + + // If depth is exhausted, or 50% of the time return a terminal + // Helps limit ridiculously large compound queries + if (depth == 0 || randomBoolean()) { + return randomTerminalQuery(stringFields, numericFields, numDocs); + } + + switch (randomIntBetween(0,5)) { + case 0: + return randomTerminalQuery(stringFields, numericFields, numDocs); + case 1: + return QueryBuilders.boolQuery().must(randomQueryBuilder(stringFields, numericFields, numDocs, depth -1)) + .filter(randomQueryBuilder(stringFields, numericFields, numDocs, depth -1)); + case 2: + return randomBoolQuery(stringFields, numericFields, numDocs, depth); + case 3: + // disabled for now because of https://issues.apache.org/jira/browse/LUCENE-6781 + //return randomBoostingQuery(stringFields, numericFields, numDocs, depth); + case 4: + return randomConstantScoreQuery(stringFields, numericFields, numDocs, depth); + case 5: + return randomDisMaxQuery(stringFields, numericFields, numDocs, depth); + default: + return randomTerminalQuery(stringFields, numericFields, numDocs); + } + } + + private static QueryBuilder randomTerminalQuery(List stringFields, List numericFields, int numDocs) { + switch (randomIntBetween(0,6)) { + case 0: + return randomTermQuery(stringFields, numDocs); + case 1: + return randomTermsQuery(stringFields, numDocs); + case 2: + return randomRangeQuery(numericFields, numDocs); + case 3: + return QueryBuilders.matchAllQuery(); + case 4: + return randomCommonTermsQuery(stringFields, numDocs); + case 5: + return randomFuzzyQuery(stringFields); + case 6: + return randomIDsQuery(); + default: + return randomTermQuery(stringFields, numDocs); + } + } + + private static String randomQueryString(int max) { + StringBuilder qsBuilder = new StringBuilder(); + + for (int i = 0; i < max; i++) { + qsBuilder.append(English.intToEnglish(randomInt(max))); + qsBuilder.append(" "); + } + + return qsBuilder.toString().trim(); + } + + private static String randomField(List fields) { + return fields.get(randomInt(fields.size() - 1)); + } + + + + private static QueryBuilder randomTermQuery(List fields, int numDocs) { + return QueryBuilders.termQuery(randomField(fields), randomQueryString(1)); + } + + private static QueryBuilder randomTermsQuery(List fields, int numDocs) { + int numTerms = randomInt(numDocs); + ArrayList terms = new ArrayList<>(numTerms); + + for (int i = 0; i < numTerms; i++) { + terms.add(randomQueryString(1)); + } + + return QueryBuilders.termsQuery(randomField(fields), terms); + } + + private static QueryBuilder randomRangeQuery(List fields, int numDocs) { + QueryBuilder q = QueryBuilders.rangeQuery(randomField(fields)); + + if (randomBoolean()) { + ((RangeQueryBuilder)q).from(randomIntBetween(0, numDocs / 2 - 1)); + } + if (randomBoolean()) { + ((RangeQueryBuilder)q).to(randomIntBetween(numDocs / 2, numDocs)); + } + + return q; + } + + private static QueryBuilder randomBoolQuery(List stringFields, List numericFields, int numDocs, int depth) { + QueryBuilder q = QueryBuilders.boolQuery(); + int numClause = randomIntBetween(0,5); + for (int i = 0; i < numClause; i++) { + ((BoolQueryBuilder)q).must(randomQueryBuilder(stringFields, numericFields,numDocs, depth -1)); + } + + numClause = randomIntBetween(0,5); + for (int i = 0; i < numClause; i++) { + ((BoolQueryBuilder)q).should(randomQueryBuilder(stringFields, numericFields,numDocs, depth -1)); + } + + numClause = randomIntBetween(0,5); + for (int i = 0; i < numClause; i++) { + ((BoolQueryBuilder)q).mustNot(randomQueryBuilder(stringFields, numericFields, numDocs, depth -1)); + } + + return q; + } + + private static QueryBuilder randomBoostingQuery(List stringFields, List numericFields, int numDocs, int depth) { + return QueryBuilders.boostingQuery( + randomQueryBuilder(stringFields, numericFields, numDocs, depth - 1), + randomQueryBuilder(stringFields, numericFields, numDocs, depth - 1)) + .boost(randomFloat()) + .negativeBoost(randomFloat()); + } + + private static QueryBuilder randomConstantScoreQuery(List stringFields, List numericFields, int numDocs, int depth) { + return QueryBuilders.constantScoreQuery(randomQueryBuilder(stringFields, numericFields, numDocs, depth - 1)); + } + + private static QueryBuilder randomCommonTermsQuery(List fields, int numDocs) { + int numTerms = randomInt(numDocs); + + QueryBuilder q = QueryBuilders.commonTermsQuery(randomField(fields), randomQueryString(numTerms)); + if (randomBoolean()) { + ((CommonTermsQueryBuilder)q).boost(randomFloat()); + } + + if (randomBoolean()) { + ((CommonTermsQueryBuilder)q).cutoffFrequency(randomFloat()); + } + + if (randomBoolean()) { + ((CommonTermsQueryBuilder)q).highFreqMinimumShouldMatch(Integer.toString(randomInt(numTerms))) + .highFreqOperator(randomBoolean() ? Operator.AND : Operator.OR); + } + + if (randomBoolean()) { + ((CommonTermsQueryBuilder)q).lowFreqMinimumShouldMatch(Integer.toString(randomInt(numTerms))) + .lowFreqOperator(randomBoolean() ? Operator.AND : Operator.OR); + } + + return q; + } + + private static QueryBuilder randomFuzzyQuery(List fields) { + + QueryBuilder q = QueryBuilders.fuzzyQuery(randomField(fields), randomQueryString(1)); + + if (randomBoolean()) { + ((FuzzyQueryBuilder)q).boost(randomFloat()); + } + + if (randomBoolean()) { + switch (randomIntBetween(0, 4)) { + case 0: + ((FuzzyQueryBuilder)q).fuzziness(Fuzziness.AUTO); + break; + case 1: + ((FuzzyQueryBuilder)q).fuzziness(Fuzziness.ONE); + break; + case 2: + ((FuzzyQueryBuilder)q).fuzziness(Fuzziness.TWO); + break; + case 3: + ((FuzzyQueryBuilder)q).fuzziness(Fuzziness.ZERO); + break; + case 4: + ((FuzzyQueryBuilder)q).fuzziness(Fuzziness.fromEdits(randomIntBetween(0,2))); + break; + default: + ((FuzzyQueryBuilder)q).fuzziness(Fuzziness.AUTO); + break; + } + } + + if (randomBoolean()) { + ((FuzzyQueryBuilder)q).maxExpansions(Math.abs(randomInt())); + } + + if (randomBoolean()) { + ((FuzzyQueryBuilder)q).prefixLength(Math.abs(randomInt())); + } + + if (randomBoolean()) { + ((FuzzyQueryBuilder)q).transpositions(randomBoolean()); + } + + return q; + } + + private static QueryBuilder randomDisMaxQuery(List stringFields, List numericFields, int numDocs, int depth) { + QueryBuilder q = QueryBuilders.disMaxQuery(); + + int numClauses = randomIntBetween(1, 10); + for (int i = 0; i < numClauses; i++) { + ((DisMaxQueryBuilder)q).add(randomQueryBuilder(stringFields, numericFields, numDocs, depth - 1)); + } + + if (randomBoolean()) { + ((DisMaxQueryBuilder)q).boost(randomFloat()); + } + + if (randomBoolean()) { + ((DisMaxQueryBuilder)q).tieBreaker(randomFloat()); + } + + return q; + } + + private static QueryBuilder randomIDsQuery() { + QueryBuilder q = QueryBuilders.idsQuery(); + + int numIDs = randomInt(100); + for (int i = 0; i < numIDs; i++) { + ((IdsQueryBuilder)q).addIds(String.valueOf(randomInt())); + } + + if (randomBoolean()) { + ((IdsQueryBuilder)q).boost(randomFloat()); + } + + return q; + } +} \ No newline at end of file diff --git a/core/src/test/java/org/elasticsearch/test/hamcrest/DoubleMatcher.java b/core/src/test/java/org/elasticsearch/test/hamcrest/DoubleMatcher.java new file mode 100644 index 00000000000..de275eaffca --- /dev/null +++ b/core/src/test/java/org/elasticsearch/test/hamcrest/DoubleMatcher.java @@ -0,0 +1,45 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.test.hamcrest; + + +public class DoubleMatcher { + + /** + * Better floating point comparisons courtesy of https://github.com/brazzy/floating-point-gui.de + * + * Snippet adapted to use doubles instead of floats + */ + public static boolean nearlyEqual(double a, double b, double epsilon) { + final double absA = Math.abs(a); + final double absB = Math.abs(b); + final double diff = Math.abs(a - b); + + if (a == b) { // shortcut, handles infinities + return true; + } else if (a == 0 || b == 0 || diff < Double.MIN_NORMAL) { + // a or b is zero or both are extremely close to it + // relative error is less meaningful here + return diff < (epsilon * Double.MIN_NORMAL); + } else { // use relative error + return diff / Math.min((absA + absB), Double.MAX_VALUE) < epsilon; + } + } +} diff --git a/docs/reference/search.asciidoc b/docs/reference/search.asciidoc index 2d8a1f8bc9a..da7d2e5ee4b 100644 --- a/docs/reference/search.asciidoc +++ b/docs/reference/search.asciidoc @@ -95,6 +95,8 @@ include::search/validate.asciidoc[] include::search/explain.asciidoc[] +include::search/profile.asciidoc[] + include::search/percolate.asciidoc[] include::search/field-stats.asciidoc[] diff --git a/docs/reference/search/profile.asciidoc b/docs/reference/search/profile.asciidoc new file mode 100644 index 00000000000..3d701ae5198 --- /dev/null +++ b/docs/reference/search/profile.asciidoc @@ -0,0 +1,601 @@ +[[search-profile]] +== Profile API + +coming[2.2.0] + +experimental[] + +The Profile API provides detailed timing information about the execution of individual components +in a query. It gives the user insight into how queries are executed at a low level so that +the user can understand why certain queries are slow, and take steps to improve their slow queries. + +The output from the Profile API is *very* verbose, especially for complicated queries executed across +many shards. Pretty-printing the response is recommended to help understand the output + +[NOTE] +--------------------- +The details provided by the Profile API directly expose Lucene class names and concepts, which means +that complete interpretation of the results require fairly advanced knowledge of Lucene. This +page attempts to give a crash-course in how Lucene executes queries so that you can use the Profile API to successfully +diagnose and debug queries, but it is only an overview. For complete understanding, please refer +to Lucene's documentation and, in places, the code. + +With that said, a complete understanding is often not required to fix a slow query. It is usually +sufficient to see that a particular component of a query is slow, and not necessarily understand why +the `advance` phase of that query is the cause, for example. +--------------------- + +[float] +=== Usage + +Any `_search` request can be profiled by adding a top-level `profile` parameter: + +[source,js] +-------------------------------------------------- +curl -XGET 'localhost:9200/_search' -d '{ + "profile": true,<1> + "query" : { + "match" : { "message" : "search test" } + } +} +-------------------------------------------------- +<1> Setting the top-level `profile` parameter to `true` will enable profiling +for the search + +This will yield the following result: + +[source,js] +-------------------------------------------------- +{ + "took": 25, + "timed_out": false, + "_shards": { + "total": 1, + "successful": 1, + "failed": 0 + }, + "hits": { + "total": 1, + "max_score": 1, + "hits": [ ... ] <1> + }, + "profile": { + "shards": [ + { + "id": "[htuC6YnSSSmKFq5UBt0YMA][test][0]", + "searches": [ + { + "query": [ + { + "query_type": "BooleanQuery", + "lucene": "message:search message:test", + "time": "15.52889800ms", + "breakdown": { + "score": 0, + "next_doc": 24495, + "match": 0, + "create_weight": 8488388, + "build_scorer": 7016015, + "advance": 0 + }, + "children": [ + { + "query_type": "TermQuery", + "lucene": "message:search", + "time": "4.938855000ms", + "breakdown": { + "score": 0, + "next_doc": 18332, + "match": 0, + "create_weight": 2945570, + "build_scorer": 1974953, + "advance": 0 + } + }, + { + "query_type": "TermQuery", + "lucene": "message:test", + "time": "0.5016660000ms", + "breakdown": { + "score": 0, + "next_doc": 0, + "match": 0, + "create_weight": 170534, + "build_scorer": 331132, + "advance": 0 + } + } + ] + } + ], + "rewrite_time": 185002, + "collector": [ + { + "name": "SimpleTopScoreDocCollector", + "reason": "search_top_hits", + "time": "2.206529000ms" + } + ] + } + ] + } + ] + } +} +-------------------------------------------------- +<1> Search results are returned, but were omitted here for brevity + +Even for a simple query, the response is relatively complicated. Let's break it down piece-by-piece before moving +to more complex examples. + +First, the overall structure of the profile response is as follows: + +[source,js] +-------------------------------------------------- +{ + "profile": { + "shards": [ + { + "id": "[htuC6YnSSSmKFq5UBt0YMA][test][0]", <1> + "searches": [ + { + "query": [...], <2> + "rewrite_time": 185002, <3> + "collector": [...] <4> + } + ] + } + ] + } +} +-------------------------------------------------- +<1> A profile is returned for each shard that participated in the response, and is identified +by a unique ID +<2> Each profile contains a section which holds details about the query execution +<3> Each profile has a single time representing the cumulative rewrite time +<4> Each profile also contains a section about the Lucene Collectors which run the search + +Because a search request may be executed against one or more shards in an index, and a search may cover +one or more indices, the top level element in the profile response is an array of `shard` objects. +Each shard object lists it's `id` which uniquely identifies the shard. The ID's format is +`[nodeID][indexName][shardID]`. + +The profile itself may consist of one or more "searches", where a search is a query executed against the underlying +Lucene index. Most Search Requests submitted by the user will only execute a single `search` against the Lucene index. +But occasionally multiple searches will be executed, such as including a global aggregation (which needs to execute +a secondary "match_all" query for the global context). + +Inside each `search` object there will be two arrays of profiled information: +a `query` array and a `collector` array. In the future, more sections may be added, such as `suggest`, `highlight`, +`aggregations`, etc + +There will also be a `rewrite` metric showing the total time spent rewriting the query (in nanoseconds). + +=== `query` Section + +The `query` section contains detailed timing of the query tree executed by Lucene on a particular shard. +The overall structure of this query tree will resemble your original Elasticsearch query, but may be slightly +(or sometimes very) different. It will also use similar but not always identical naming. Using our previous +`term` query example, let's analyze the `query` section: + +[source,js] +-------------------------------------------------- +"query": [ + { + "query_type": "BooleanQuery", + "lucene": "message:search message:test", + "time": "15.52889800ms", + "breakdown": {...}, + "children": [ + { + "query_type": "TermQuery", + "lucene": "message:search", + "time": "4.938855000ms", + "breakdown": {...} + }, + { + "query_type": "TermQuery", + "lucene": "message:test", + "time": "0.5016660000ms", + "breakdown": {...} + } + ] + } +] +-------------------------------------------------- +<1> The breakdown timings are omitted for simplicity + +Based on the profile structure, we can see that our `match` query was rewritten by Lucene into a BooleanQuery with two +clauses (both holding a TermQuery). The `"query_type"` field displays the Lucene class name, and often aligns with +the equivalent name in Elasticsearch. The `"lucene"` field displays the Lucene explanation text for the query, and +is made available to help differentiating between parts of your query (e.g. both `"message:search"` and `"message:test"` +are TermQuery's and would appear identical otherwise. + +The `"time"` field shows that this query took ~15ms for the entire BooleanQuery to execute. The recorded time is inclusive +of all children. + +The `"breakdown"` field will give detailed stats about how the time was spent, we'll look at +that in a moment. Finally, the `"children"` array lists any sub-queries that may be present. Because we searched for two +values ("search test"), our BooleanQuery holds two children TermQueries. They have identical information (query_type, time, +breakdown, etc). Children are allowed to have their own children. + +==== Timing Breakdown + +The `breakdown` component lists detailed timing statistics about low-level Lucene execution: + +[source,js] +-------------------------------------------------- +"breakdown": { + "score": 0, + "next_doc": 24495, + "match": 0, + "create_weight": 8488388, + "build_scorer": 7016015, + "advance": 0 + +} +-------------------------------------------------- + +Timings are listed in wall-clock nanoseconds and are not normalized at all. All caveats about the overall +`time` apply here. The intention of the breakdown is to give you a feel for A) what machinery in Lucene is +actually eating time, and B) the magnitude of differences in times between the various components. Like the overall time, +the breakdown is inclusive of all children times. + +The meaning of the stats are as follows: + +[float] +=== All parameters: + +[horizontal] + +`create_weight`:: + + A Query in Lucene must be capable of reuse across multiple IndexSearchers (think of it as the engine that + executes a search against a specific Lucene Index). This puts Lucene in a tricky spot, since many queries + need to accumulate temporary state/statistics associated with the index it is being used against, but the + Query contract mandates that it must be immutable. + + To get around this, Lucene asks each query to generate a Weight object which acts as a temporary context + object to hold state associated with this particular (IndexSearcher, Query) tuple. The `weight` metric + shows how long this process takes + +`build_scorer`:: + + This parameter shows how long it takes to build a Scorer for the query. A Scorer is the mechanism that + iterates over matching documents generates a score per-document (e.g. how well does "foo" match the document?). + Note, this records the time required to generate the Scorer object, not actuall score the documents. Some + queries have faster or slower initialization of the Scorer, depending on optimizations, complexity, etc. + + This may also showing timing associated with caching, if enabled and/or applicable for the query + +`next_doc`:: + + The Lucene method `next_doc` returns Doc ID of the next document matching the query. This statistic shows + the time it takes to determine which document is the next match, a process that varies considerably depending + on the nature of the query. Next_doc is a specialized form of advance() which is more convenient for many + queries in Lucene. It is equivalent to advance(docId() + 1) + +`advance`:: + + `advance` is the "lower level" version of next_doc: it serves the same purpose of finding the next matching + doc, but requires the calling query to perform extra tasks such as identifying and moving past skips, etc. + However, not all queries can use next_doc, so `advance` is also timed for those queries. + + Conjunctions (e.g. `must` clauses in a boolean) are typical consumers of `advance` + +`matches`:: + + Some queries, such as phrase queries, match documents using a "Two Phase" process. First, the document is + "approximately" matched, and if it matches approximately, it is checked a second time with a more rigorous + (and expensive) process. The second phase verification is what the `matches` statistic measures. + + For example, a phrase query first checks a document approximately by ensuring all terms in the phrase are + present in the doc. If all the terms are present, it then executes the second phase verification to ensure + the terms are in-order to form the phrase, which is relatively more expensive than just checking for presence + of the terms. + + Because this two-phase process is only used by a handful of queries, the `metric` statistic will often be zero + +`score`:: + + This records the time taken to score a particular document via it's Scorer + +=== `collectors` Section + +The Collectors portion of the response shows high-level execution details. Lucene works by defining a "Collector" +which is responsible for coordinating the traversal, scoring and collection of matching documents. Collectors +are also how a single query can record aggregation results, execute unscoped "global" queries, execute post-query +filters, etc. + +Looking at the previous example: + +[source,js] +-------------------------------------------------- +"collector": [ + { + "name": "SimpleTopScoreDocCollector", + "reason": "search_top_hits", + "time": "2.206529000ms" + } +] +-------------------------------------------------- + +We see a single collector named `SimpleTopScoreDocCollector`. This is the default "scoring and sorting" Collector +used by Elasticsearch. The `"reason"` field attempts to give an plain english description of the class name. The +`"time` is similar to the time in the Query tree: a wall-clock time inclusive of all children. Similarly, `children` lists +all sub-collectors. + +It should be noted that Collector times are **independent** from the Query times. They are calculated, combined +and normalized independently! Due to the nature of Lucene's execution, it is impossible to "merge" the times +from the Collectors into the Query section, so they are displayed in separate portions. + +For reference, the various collector reason's are: + +[horizontal] +`search_sorted`:: + + A collector that scores and sorts documents. This is the most common collector and will be seen in most + simple searches + +`search_count`:: + + A collector that only counts the number of documents that match the query, but does not fetch the source. + This is seen when `size: 0` or `search_type=count` is specified + +`search_terminate_after_count`:: + + A collector that terminates search execution after `n` matching documents have been found. This is seen + when the `terminate_after_count` query parameter has been specified + +`search_min_score`:: + + A collector that only returns matching documents that have a score greater than `n`. This is seen when + the top-level paramenter `min_score` has been specified. + +`search_multi`:: + + A collector that wraps several other collectors. This is seen when combinations of search, aggregations, + global aggs and post_filters are combined in a single search. + +`search_timeout`:: + + A collector that halts execution after a specified period of time. This is seen when a `timeout` top-level + parameter has been specified. + +`aggregation`:: + + A collector that Elasticsearch uses to run aggregations against the query scope. A single `aggregation` + collector is used to collect documents for *all* aggregations, so you will see a list of aggregations + in the name rather. + +`global_aggregation`:: + + A collector that executes an aggregation against the global query scope, rather than the specified query. + Because the global scope is necessarily different from the executed query, it must execute it's own + match_all query (which you will see added to the Query section) to collect your entire dataset + + + +=== `rewrite` Section + + All queries in Lucene undergo a "rewriting" process. A query (and its sub-queries) may be rewritten one or + more times, and the process continues until the query stops changing. This process allows Lucene to perform + optimizations, such as removing redundant clauses, replacing one query for a more efficient execution path, + etc. For example a Boolean -> Boolean -> TermQuery can be rewritten to a TermQuery, because all the Booleans + are unnecessary in this case. + + The rewriting process is complex and difficult to display, since queries can change drastically. Rather than + showing the intermediate results, the total rewrite time is simply displayed as a value (in nanoseconds). This + value is cumulative and contains the total time for all queries being rewritten. + +=== A more complex example + +To demonstrate a slightly more complex query and the associated results, we can profile the following query: + +[source,js] +-------------------------------------------------- +GET /test/_search +{ + "profile": true, + "query": { + "term": { + "message": { + "value": "search" + } + } + }, + "aggs": { + "non_global_term": { + "terms": { + "field": "agg" + }, + "aggs": { + "second_term": { + "terms": { + "field": "sub_agg" + } + } + } + }, + "another_agg": { + "cardinality": { + "field": "aggB" + } + }, + "global_agg": { + "global": {}, + "aggs": { + "my_agg2": { + "terms": { + "field": "globalAgg" + } + } + } + } + }, + "post_filter": { + "term": { + "my_field": "foo" + } + } +} +-------------------------------------------------- + +This example has: + +- A query +- A scoped aggregation +- A global aggregation +- A post_filter + +And the response: + + +[source,js] +-------------------------------------------------- +{ + "profile": { + "shards": [ + { + "id": "[P6-vulHtQRWuD4YnubWb7A][test][0]", + "searches": [ + { + "query": [ + { + "query_type": "TermQuery", + "lucene": "my_field:foo", + "time": "0.4094560000ms", + "breakdown": { + "score": 0, + "next_doc": 0, + "match": 0, + "create_weight": 31584, + "build_scorer": 377872, + "advance": 0 + } + }, + { + "query_type": "TermQuery", + "lucene": "message:search", + "time": "0.3037020000ms", + "breakdown": { + "score": 0, + "next_doc": 5936, + "match": 0, + "create_weight": 185215, + "build_scorer": 112551, + "advance": 0 + } + } + ], + "rewrite_time": 7208, + "collector": [ + { + "name": "MultiCollector", + "reason": "search_multi", + "time": "1.378943000ms", + "children": [ + { + "name": "FilteredCollector", + "reason": "search_post_filter", + "time": "0.4036590000ms", + "children": [ + { + "name": "SimpleTopScoreDocCollector", + "reason": "search_top_hits", + "time": "0.006391000000ms" + } + ] + }, + { + "name": "BucketCollector: [[non_global_term, another_agg]]", + "reason": "aggregation", + "time": "0.9546020000ms" + } + ] + } + ] + }, + { + "query": [ + { + "query_type": "MatchAllDocsQuery", + "lucene": "*:*", + "time": "0.04829300000ms", + "breakdown": { + "score": 0, + "next_doc": 3672, + "match": 0, + "create_weight": 6311, + "build_scorer": 38310, + "advance": 0 + } + } + ], + "rewrite_time": 1067, + "collector": [ + { + "name": "GlobalAggregator: [global_agg]", + "reason": "aggregation_global", + "time": "0.1226310000ms" + } + ] + } + ] + } + ] + } +} +-------------------------------------------------- + +As you can see, the output is significantly verbose from before. All the major portions of the query are +represented: + +1. The first `TermQuery` (message:search) represents the main `term` query +2. The second `TermQuery` (my_field:foo) represents the `post_filter` query +3. There is a `MatchAllDocsQuery` (*:*) query which is being executed as a second, distinct search. This was +not part of the query specified by the user, but is auto-generated by the global aggregation to provide a global query scope + +The Collector tree is fairly straightforward, showing how a single MultiCollector wraps a FilteredCollector +to execute the post_filter (and in turn wraps the normal scoring SimpleCollector), a BucketCollector to run +all scoped aggregations. In the MatchAll search, there is a single GlobalAggregator to run the global aggregation. + +=== Performance Notes + +Like any profiler, the Profile API introduce a non-negligible overhead to query execution. The act of instrumenting +low-level method calls such as `advance` and `next_doc` can be fairly expensive, since these methods are called +in tight loops. Therefore, profiling should not be enabled in production settings by default, and should not +be compared against non-profiled query times. Profiling is just a diagnostic tool. + +There are also cases where special Lucene optimizations are disabled, since they are not amenable to profiling. This +could cause some queries to report larger relative times than their non-profiled counterparts, but in general should +not have a drastic effect compared to other components in the profiled query. + +=== Limitations + +- Profiling statistics are currently not available for suggestions, highlighting, `dfs_query_then_fetch` +- Detailed breakdown for aggregations is not currently available past the high-level overview provided +from the Collectors +- The Profiler is still highly experimental. The Profiler is instrumenting parts of Lucene that were +never designed to be exposed in this manner, and so all results should be viewed as a best effort to provide detailed +diagnostics. We hope to improve this over time. If you find obviously wrong numbers, strange query structures or +other bugs, please report them! + +=== Understanding MultiTermQuery output + +A special note needs to be made about the `MultiTermQuery` class of queries. This includes wildcards, regex and fuzzy +queries. These queries emit very verbose responses, and are not overly structured. + +Essentially, these queries rewrite themselves on a per-segment basis. If you imagine the wildcard query `b*`, it technically +can match any token that begins with the letter "b". It would be impossible to enumerate all possible combinations, +so Lucene rewrites the query in context of the segment being evaluated. E.g. one segment may contain the tokens +`[bar, baz]`, so the query rewrites to a BooleanQuery combination of "bar" and "baz". Another segment may only have the +token `[bakery]`, so query rewrites to a single TermQuery for "bakery". + +Due to this dynamic, per-segment rewriting, the clean tree structure becomes distorted and no longer follows a clean +"lineage" showing how one query rewrites into the next. At present time, all we can do is apologize, and suggest you +collapse the details for that query's children if it is too confusing. Luckily, all the timing statistics are correct, +just not the physical layout in the response, so it is sufficient to just analyze the top-level MultiTermQuery and +ignore it's children if you find the details too tricky to interpret. + +Hopefully this will be fixed in future iterations, but it is a tricky problem to solve and still in-progress :) \ No newline at end of file diff --git a/test-framework/src/main/java/org/elasticsearch/test/TestSearchContext.java b/test-framework/src/main/java/org/elasticsearch/test/TestSearchContext.java index 468b1877250..51fb0c905f4 100644 --- a/test-framework/src/main/java/org/elasticsearch/test/TestSearchContext.java +++ b/test-framework/src/main/java/org/elasticsearch/test/TestSearchContext.java @@ -35,7 +35,6 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; -import org.elasticsearch.index.cache.query.QueryCache; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MappedFieldType; @@ -60,6 +59,8 @@ import org.elasticsearch.search.internal.ScrollContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.lookup.SearchLookup; +import org.elasticsearch.search.profile.Profiler; +import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rescore.RescoreSearchContext; import org.elasticsearch.search.suggest.SuggestionSearchContext; @@ -293,7 +294,7 @@ public class TestSearchContext extends SearchContext { } public void setSearcher(Engine.Searcher searcher) { - this.searcher = new ContextIndexSearcher(this, searcher); + this.searcher = new ContextIndexSearcher(searcher, indexService.cache().query(), indexShard.getQueryCachingPolicy()); } @Override @@ -660,8 +661,11 @@ public class TestSearchContext extends SearchContext { public void copyContextAndHeadersFrom(HasContextAndHeaders other) {} @Override - public Map, Collector> queryCollectors() {return queryCollectors;} + public Profilers getProfilers() { + return null; // no profiling + } @Override - public QueryCache getQueryCache() { return indexService.cache().query();} + public Map, Collector> queryCollectors() {return queryCollectors;} + }